Proposal: Generic WAL logical messages
Hello,
I would like to introduce concept of generic WAL logical messages.
These messages just create WAL record with arbitrary data as specified
by user. For standard WAL reply, these are basically noop, but in
logical decoding they are be decoded and the special callback of the
output plugin is be called for them.
These messages can be both transactional (decoded on commit) or
non-transactional (decoded immediately). Each message has prefix to
differentiate between individual plugins. The prefix has to be
registered exactly once (in similar manner as security label providers)
to avoid conflicts between plugins.
There are three main use-cases for these:
a) reliable communication between two nodes in multi-master setup - WAL
already handles correctly the recovery etc so we don't have to invent
new complex code to handle crashes in middle of operations
b) out of order messaging in logical replication - if you want to send
something to other node immediately without having to wait for commit
for example to acquire remote lock, this can be used for that
c) "queue tables" - combination of this feature (there is SQL interface)
and before triggers can be used to create tables for which all inserts
only go to the WAL so they can behave like queue without having to store
the data in the table itself (kind of the opposite of unlogged tables)
Initial implementation of this proposal is attached.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
WAL-Messages-2016-01-01.patchbinary/octet-stream; name=WAL-Messages-2016-01-01.patchDownload
>From b66a78366a7234842d8c0a351d090fc1855d0da3 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Fri, 1 Jan 2016 04:27:32 +0100
Subject: [PATCH] WAL Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/messages.out | 56 +++++++++++++++++
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 21 ++++++-
doc/src/sgml/func.sgml | 42 +++++++++++++
doc/src/sgml/logicaldecoding.sgml | 22 +++++++
src/backend/access/rmgrdesc/standbydesc.c | 8 +++
src/backend/replication/logical/decode.c | 19 ++++++
src/backend/replication/logical/logical.c | 38 ++++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++++
src/backend/replication/logical/reorderbuffer.c | 73 ++++++++++++++++++++++
src/backend/storage/ipc/standby.c | 82 +++++++++++++++++++++++++
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/output_plugin.h | 13 ++++
src/include/replication/reorderbuffer.h | 21 +++++++
src/include/storage/standby.h | 15 +++++
17 files changed, 460 insertions(+), 2 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index a362e69..8fdcfbc 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl rewrite toast permissions decoding_in_xact decoding_into_rel \
- binary prepared replorigin
+ binary prepared replorigin messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..d256851
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_send_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_send_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_send_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_send_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT regexp_replace(data, 'lsn: [^ ]+', '') FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ regexp_replace
+-------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..7f462b8
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_send_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_send_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_send_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_send_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT regexp_replace(data, 'lsn: [^ ]+', '') FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 80fc5f4..064bb5f 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -63,11 +63,15 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
{
- /* other plugins can perform things here */
+ RegisterStandbyMsgPrefix("test");
}
/* specify output plugin callbacks */
@@ -82,6 +86,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +476,17 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: lsn: %X/%X transactional: %d prefix: %s, sz: %zu content:",
+ (uint32)(lsn >> 32), (uint32)lsn, transactional, prefix,
+ sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 8ef9fce..e0fe571 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -17580,6 +17580,48 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-send-message-text">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text standby message. This can used by logical decoding for
+ sending generic messages. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-send-message-bytea">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary standby message. This can used by logical decoding for
+ sending generic messages. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 02794cc..eae3639 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,27 @@ typedef bool (*LogicalDecodeChangeCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Genenric Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a standby message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/standbydesc.c b/src/backend/access/rmgrdesc/standbydesc.c
index 3d35f38..ecf4f6e 100644
--- a/src/backend/access/rmgrdesc/standbydesc.c
+++ b/src/backend/access/rmgrdesc/standbydesc.c
@@ -58,6 +58,14 @@ standby_desc(StringInfo buf, XLogReaderState *record)
standby_desc_running_xacts(buf, xlrec);
}
+ else if (info == XLOG_STANDBY_MESSAGE)
+ {
+ xl_standby_message *xlrec = (xl_standby_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
}
const char *
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 9f60687..008301a 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -298,6 +298,25 @@ DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
break;
case XLOG_STANDBY_LOCK:
break;
+ case XLOG_STANDBY_MESSAGE:
+ {
+ xl_standby_message *message = (xl_standby_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ break;
+ else if(!message->transactional &&
+ (SnapBuildCurrentState(ctx->snapshot_builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ break;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr, message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ break;
+ }
default:
elog(ERROR, "unexpected RM_STANDBY_ID record type: %u", info);
}
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e14a3e..d4d58d9 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn; /* beginning of commit record */
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional,
+ prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 012987a..a76f8c4 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -23,6 +23,8 @@
#include "access/xlog_internal.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -42,6 +44,7 @@
#include "replication/logicalfuncs.h"
#include "storage/fd.h"
+#include "storage/standby.h"
/* private date for writing out data */
typedef struct DecodingOutputState
@@ -517,3 +520,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
return ret;
}
+
+
+/*
+ * SQL function returning the changestream in binary, only peeking ahead.
+ */
+Datum
+pg_logical_send_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogStandbyMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_send_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_send_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index cdc7bd7..4ee7bb9 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,11 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -599,6 +604,39 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz,
+ const char *msg)
+{
+ ReorderBufferTXN *txn = NULL;
+
+ if (transactional)
+ {
+ ReorderBufferChange *change;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ Assert(xid != InvalidTransactionId);
+ Assert(txn != NULL);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.transactional = true;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+ }
+ else
+ {
+ rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg);
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1465,6 +1503,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.transactional,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2117,6 +2163,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2354,6 +2415,18 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
data += len;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = pstrdup(data);
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, data + prefix_size,
+ message_size);
+
+ data += prefix_size + message_size;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..9e2d01b 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -28,6 +28,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/standby.h"
+#include "utils/memutils.h"
#include "utils/ps_status.h"
#include "utils/timeout.h"
#include "utils/timestamp.h"
@@ -39,6 +40,8 @@ int max_standby_streaming_delay = 30 * 1000;
static List *RecoveryLockList;
+static List *StandbyMsgPrefixList = NIL;
+
static void ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
ProcSignalReason reason);
static void ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid);
@@ -795,6 +798,10 @@ standby_redo(XLogReaderState *record)
ProcArrayApplyRecoveryInfo(&running);
}
+ else if (info == XLOG_STANDBY_MESSAGE)
+ {
+ /* Only interesting to logical decoding. Check decode.c */
+ }
else
elog(PANIC, "standby_redo: unknown op code %u", info);
}
@@ -1038,3 +1045,78 @@ LogAccessExclusiveLockPrepare(void)
*/
(void) GetTopTransactionId();
}
+
+XLogRecPtr
+LogStandbyMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ ListCell *lc;
+ bool found = false;
+ xl_standby_message xlrec;
+
+ /* Check if the provided prefix is known to us. */
+ foreach(lc, StandbyMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("standby message prefix \"%s\" is not registered",
+ prefix)));
+
+ /*
+ * Force xid to be allocated if we're sending a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfStandbyMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_STANDBY_ID, XLOG_STANDBY_MESSAGE);
+}
+
+void
+RegisterStandbyMsgPrefix(const char *prefix)
+{
+ ListCell *lc;
+ bool found = false;
+ MemoryContext oldcxt;
+
+ /* Check for duplicit registrations. */
+ foreach(lc, StandbyMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("standby message prefix \"%s\" is already registered",
+ prefix)));
+
+ oldcxt = MemoryContextSwitchTo(TopMemoryContext);
+ StandbyMsgPrefixList = lappend(StandbyMsgPrefixList, pstrdup(prefix));
+ MemoryContextSwitchTo(oldcxt);
+}
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d8640db..8871207 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5215,6 +5215,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_text _null_ _null_ _null_ ));
+DESCR("send a textual message");
+DATA(insert OID = 3578 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_bytea _null_ _null_ _null_ ));
+DESCR("send a binary message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index 03fbc4a..bf94193 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_send_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_send_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 17c3de2..d3df560 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for messages sent by C code.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index c9a11db..8d2fca7 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -51,6 +51,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -95,6 +96,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ struct
+ {
+ bool transactional;
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -271,6 +280,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -297,6 +315,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -347,6 +366,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/storage/standby.h b/src/include/storage/standby.h
index 40b329b..3f036cb 100644
--- a/src/include/storage/standby.h
+++ b/src/include/storage/standby.h
@@ -56,6 +56,7 @@ extern void StandbyReleaseOldLocks(int nxids, TransactionId *xids);
*/
#define XLOG_STANDBY_LOCK 0x00
#define XLOG_RUNNING_XACTS 0x10
+#define XLOG_STANDBY_MESSAGE 0x20
typedef struct xl_standby_locks
{
@@ -80,6 +81,17 @@ typedef struct xl_running_xacts
#define MinSizeOfXactRunningXacts offsetof(xl_running_xacts, xids)
+typedef struct xl_standby_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_standby_message;
+
+#define SizeOfStandbyMessage (offsetof(xl_standby_message, message))
/* Recovery handlers for the Standby Rmgr (RM_STANDBY_ID) */
extern void standby_redo(XLogReaderState *record);
@@ -116,5 +128,8 @@ extern void LogAccessExclusiveLock(Oid dbOid, Oid relOid);
extern void LogAccessExclusiveLockPrepare(void);
extern XLogRecPtr LogStandbySnapshot(void);
+extern XLogRecPtr LogStandbyMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+extern void RegisterStandbyMsgPrefix(const char *prefix);
#endif /* STANDBY_H */
--
1.9.1
Petr Jelinek wrote:
These messages just create WAL record with arbitrary data as specified by
user. For standard WAL reply, these are basically noop, but in logical
decoding they are be decoded and the special callback of the output plugin
is be called for them.
I had a quick look at this and I couldn't find anything worthy of
comment -- seems a reasonably straightforward addition to stuff that's
mostly already there.
Process-wise, I don't understand why is Andres mentioned as co-author.
Did he actually wrote part of the patch, or advised on the design?
Who is reviewing the patch?
--
�lvaro Herrera http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Process-wise, I don't understand why is Andres mentioned as co-author.
Did he actually wrote part of the patch, or advised on the design?
Who is reviewing the patch?
It's extracted & extended from BDR, where I added that feature (to
implement distributed locking).
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 1 January 2016 at 03:59, Petr Jelinek <petr@2ndquadrant.com> wrote:
I would like to introduce concept of generic WAL logical messages.
Couple of points...
* Genenric misspelled
* You call them "logical messages" here, but standby messages in code. But
they only apply to logical decoding, so "logical message" seems a better
name. Can we avoid calling them "messages" cos that will get confusing.
For standard WAL reply, these are basically noop
We should document that.
These messages can be both transactional (decoded on commit) or
non-transactional (decoded immediately). Each message has prefix to
differentiate between individual plugins. The prefix has to be registered
exactly once (in similar manner as security label providers) to avoid
conflicts between plugins.
I'm not sure what "transactional" means, nor is that documented.
(Conversely, I think "immediate" fairly clear)
Are they fired only on commit? (Guess so)
Are they fired in the original order, if multiple messages in same
transaction? (Hope so)
Are they fired as they come in the original message sequence, or before
anything else or after everything else? For example, cache invalidation
messages are normally fired right at the end of a transaction, no matter
when they were triggered.
--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/>
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
On 2016-01-07 17:22, Simon Riggs wrote:
* You call them "logical messages" here, but standby messages in code.
But they only apply to logical decoding, so "logical message" seems a
better name. Can we avoid calling them "messages" cos that will get
confusing.
Yes it's slightly confusing, the "Standby" in the code is mostly for
consistency with other "Standby*" stuff in neighbouring code, but that
can be changed. I don't have better name than "messages" though,
"records" is too generic.
For standard WAL reply, these are basically noop
We should document that.
Okay.
These messages can be both transactional (decoded on commit) or
non-transactional (decoded immediately). Each message has prefix to
differentiate between individual plugins. The prefix has to be
registered exactly once (in similar manner as security label
providers) to avoid conflicts between plugins.I'm not sure what "transactional" means, nor is that documented.
(Conversely, I think "immediate" fairly clear)Are they fired only on commit? (Guess so)
Are they fired in the original order, if multiple messages in same
transaction? (Hope so)
Are they fired as they come in the original message sequence, or before
anything else or after everything else? For example, cache invalidation
messages are normally fired right at the end of a transaction, no matter
when they were triggered.
Transnational message is added to the stream same way as any DML
operation is and has same properties (processed on commit, original
order, duplicate messages are delivered as they are).
The immediate as you say is obvious, they get to logical decoding
immediately without dealing with any regards to what's happening around
(wal order is still preserved though).
Will make this clearer in the docs.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
here is updated version of this patch, calling the messages logical
(decoding) messages consistently everywhere and removing any connection
to standby messages. Moving this to it's own module gave me place to
write some brief explanation about this so the code documentation has
hopefully improved as well.
The functionality itself didn't change.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2015-01-22.patchapplication/x-patch; name=logical-messages-2015-01-22.patchDownload
From c19d7274091baee4a523333b1fa0ac684ace4cc9 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Fri, 22 Jan 2016 20:15:22 +0100
Subject: [PATCH] Logical Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/test_decoding.c | 22 +++-
doc/src/sgml/func.sgml | 42 ++++++++
doc/src/sgml/logicaldecoding.sgml | 22 ++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 33 ++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 37 +++++++
src/backend/replication/logical/logical.c | 38 +++++++
src/backend/replication/logical/logicalfuncs.c | 27 +++++
src/backend/replication/logical/message.c | 132 ++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 73 +++++++++++++
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 +
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 42 ++++++++
src/include/replication/output_plugin.h | 13 +++
src/include/replication/reorderbuffer.h | 21 ++++
20 files changed, 514 insertions(+), 5 deletions(-)
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index a362e69..8fdcfbc 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl rewrite toast permissions decoding_in_xact decoding_into_rel \
- binary prepared replorigin
+ binary prepared replorigin messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..f655355 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,11 +64,15 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
{
- /* other plugins can perform things here */
+ RegisterLogicalMsgPrefix("test");
}
/* specify output plugin callbacks */
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,17 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: lsn: %X/%X transactional: %d prefix: %s, sz: %zu content:",
+ (uint32)(lsn >> 32), (uint32)lsn, transactional, prefix,
+ sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 4d2b88f..d55a2b1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -17639,6 +17639,48 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-send-message-text">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-send-message-bytea">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1ae5eb6..1d28de0 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,27 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..739ee10
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 88c3a49..110f958 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -115,6 +117,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -431,6 +437,37 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+ else if(!message->transactional &&
+ (SnapBuildCurrentState(ctx->snapshot_builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr, message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..998fb79 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn; /* beginning of commit record */
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional,
+ prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..2eb26d4 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function returning the changestream in binary, only peeking ahead.
+ */
+Datum
+pg_logical_send_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_send_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_send_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..e5f4feb
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,132 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/* List of registered logical message prefixes. */
+static List *LogicalMsgPrefixList = NIL;
+
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ ListCell *lc;
+ bool found = false;
+ xl_logical_message xlrec;
+
+ /* Check if the provided prefix is known to us. */
+ foreach(lc, LogicalMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("standby message prefix \"%s\" is not registered",
+ prefix)));
+
+ /*
+ * Force xid to be allocated if we're sending a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+void
+RegisterLogicalMsgPrefix(const char *prefix)
+{
+ ListCell *lc;
+ bool found = false;
+ MemoryContext oldcxt;
+
+ /* Check for duplicit registrations. */
+ foreach(lc, LogicalMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("logical decoding message prefix \"%s\" is already registered",
+ prefix)));
+
+ oldcxt = MemoryContextSwitchTo(TopMemoryContext);
+ LogicalMsgPrefixList = lappend(LogicalMsgPrefixList, pstrdup(prefix));
+ MemoryContextSwitchTo(oldcxt);
+}
+
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 78acced..99e9ec7 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,11 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -599,6 +604,39 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz,
+ const char *msg)
+{
+ ReorderBufferTXN *txn = NULL;
+
+ if (transactional)
+ {
+ ReorderBufferChange *change;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ Assert(xid != InvalidTransactionId);
+ Assert(txn != NULL);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.transactional = true;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+ }
+ else
+ {
+ rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg);
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1465,6 +1503,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.transactional,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2117,6 +2163,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2354,6 +2415,18 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
data += len;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = pstrdup(data);
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, data + prefix_size,
+ message_size);
+
+ data += prefix_size + message_size;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 244aa4d..8da6fae 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5066,6 +5066,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_text _null_ _null_ _null_ ));
+DESCR("send a textual message");
+DATA(insert OID = 3578 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_bytea _null_ _null_ _null_ ));
+DESCR("send a binary message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..cd3373e 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_send_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_send_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..e4223f4
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,42 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+extern void RegisterLogicalMsgPrefix(const char *prefix);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 2abee0a..f51f5c1 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -51,6 +51,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -95,6 +96,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ struct
+ {
+ bool transactional;
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -271,6 +280,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -297,6 +315,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -347,6 +366,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
--
1.9.1
Hi, Petr!
On Sat, Jan 23, 2016 at 1:22 AM, Petr Jelinek <petr@2ndquadrant.com> wrote:
here is updated version of this patch, calling the messages logical
(decoding) messages consistently everywhere and removing any connection to
standby messages. Moving this to it's own module gave me place to write
some brief explanation about this so the code documentation has hopefully
improved as well.The functionality itself didn't change.
I'd like to mention that there is my upcoming patch which is named generic
WAL records.
*/messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com
</messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com>*
But it has to be distinct feature from your generic WAL logical messages.
Theoretically, we could have generic messages with arbitrary content and
both having custom WAL reply function and being decoded by output plugin.
But custom WAL reply function would let extension bug break recovery,
archiving and physical replication. And that doesn't seem to be acceptable.
This is why we have to develop these as separate features.
Should we think more about naming? Does two kinds of generic records
confuse people?
------
Alexander Korotkov
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company
On 29 January 2016 at 21:11, Alexander Korotkov <a.korotkov@postgrespro.ru>
wrote:
Hi, Petr!
On Sat, Jan 23, 2016 at 1:22 AM, Petr Jelinek <petr@2ndquadrant.com>
wrote:here is updated version of this patch, calling the messages logical
(decoding) messages consistently everywhere and removing any connection to
standby messages. Moving this to it's own module gave me place to write
some brief explanation about this so the code documentation has hopefully
improved as well.The functionality itself didn't change.
I'd like to mention that there is my upcoming patch which is named generic
WAL records.
*/messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com
</messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com>*
But it has to be distinct feature from your generic WAL logical messages.
Theoretically, we could have generic messages with arbitrary content and
both having custom WAL reply function and being decoded by output plugin.
But custom WAL reply function would let extension bug break recovery,
archiving and physical replication. And that doesn't seem to be acceptable.
This is why we have to develop these as separate features.Should we think more about naming? Does two kinds of generic records
confuse people?
Logical messages
Generic WAL records
Seems like I can tell them apart. Worth checking, but I think we're OK.
--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/>
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
On Sat, Jan 30, 2016 at 11:58 AM, Simon Riggs <simon@2ndquadrant.com> wrote:
On 29 January 2016 at 21:11, Alexander Korotkov <a.korotkov@postgrespro.ru
wrote:
Hi, Petr!
On Sat, Jan 23, 2016 at 1:22 AM, Petr Jelinek <petr@2ndquadrant.com>
wrote:here is updated version of this patch, calling the messages logical
(decoding) messages consistently everywhere and removing any connection to
standby messages. Moving this to it's own module gave me place to write
some brief explanation about this so the code documentation has hopefully
improved as well.The functionality itself didn't change.
I'd like to mention that there is my upcoming patch which is named
generic WAL records.
*/messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com
</messages/by-id/CAPpHfdsXwZmojm6Dx+TJnpYk27kT4o7Ri6X_4OSWcByu1Rm+VA@mail.gmail.com>*
But it has to be distinct feature from your generic WAL logical messages.
Theoretically, we could have generic messages with arbitrary content and
both having custom WAL reply function and being decoded by output plugin.
But custom WAL reply function would let extension bug break recovery,
archiving and physical replication. And that doesn't seem to be acceptable.
This is why we have to develop these as separate features.Should we think more about naming? Does two kinds of generic records
confuse people?Logical messages
Generic WAL records
Seems like I can tell them apart. Worth checking, but I think we're OK.
I was worrying because topic name is "Generic WAL logical messages". But if
we name them just "Logical messages" then it's OK for me.
------
Alexander Korotkov
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company
On 1 February 2016 at 09:45, Alexander Korotkov
<a.korotkov@postgrespro.ru> wrote:
On Sat, Jan 30, 2016 at 11:58 AM, Simon Riggs <simon@2ndquadrant.com> wrote:
On 29 January 2016 at 21:11, Alexander Korotkov
<a.korotkov@postgrespro.ru> wrote:Should we think more about naming? Does two kinds of generic records
confuse people?Logical messages
Generic WAL records
Seems like I can tell them apart. Worth checking, but I think we're OK.
I was worrying because topic name is "Generic WAL logical messages". But if
we name them just "Logical messages" then it's OK for me.
Yeah the patch talks about logical messages, I use different title in
mailing list and CF to make it more clear on first sight what this
actually is technically.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 23.01.2016 01:22, Petr Jelinek wrote:
Hi,
here is updated version of this patch, calling the messages logical
(decoding) messages consistently everywhere and removing any connection
to standby messages. Moving this to it's own module gave me place to
write some brief explanation about this so the code documentation has
hopefully improved as well.The functionality itself didn't change.
Hello,
It seems that you forgot regression tests for test_decoding. There is an
entry in test_decoding/Makefile, but there are not files
sql/messages.sql and expected/messages.out. However they are included in
the first version of the patch.
--
Artur Zakirov
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hello,
It seems that you forgot regression tests for test_decoding. There is an
entry in test_decoding/Makefile, but there are not files
sql/messages.sql and expected/messages.out. However they are included in
the first version of the patch.
Hi, yes, git add missing.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-02-24.patchapplication/x-patch; name=logical-messages-2016-02-24.patchDownload
From 55a771c4770b73b5bedac7cd91bd0a50b0d6da45 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/messages.out | 56 ++++++++++
contrib/test_decoding/sql/messages.sql | 17 +++
contrib/test_decoding/test_decoding.c | 22 +++-
doc/src/sgml/func.sgml | 42 ++++++++
doc/src/sgml/logicaldecoding.sgml | 22 ++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 33 ++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 37 +++++++
src/backend/replication/logical/logical.c | 38 +++++++
src/backend/replication/logical/logicalfuncs.c | 27 +++++
src/backend/replication/logical/message.c | 132 ++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 73 +++++++++++++
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 +
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 42 ++++++++
src/include/replication/output_plugin.h | 13 +++
src/include/replication/reorderbuffer.h | 21 ++++
22 files changed, 587 insertions(+), 5 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index a362e69..8fdcfbc 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl rewrite toast permissions decoding_in_xact decoding_into_rel \
- binary prepared replorigin
+ binary prepared replorigin messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..d256851
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_send_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_send_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_send_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_send_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT regexp_replace(data, 'lsn: [^ ]+', '') FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ regexp_replace
+-------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..7f462b8
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_send_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_send_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_send_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_send_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT regexp_replace(data, 'lsn: [^ ]+', '') FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..f655355 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,11 +64,15 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
{
- /* other plugins can perform things here */
+ RegisterLogicalMsgPrefix("test");
}
/* specify output plugin callbacks */
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,17 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: lsn: %X/%X transactional: %d prefix: %s, sz: %zu content:",
+ (uint32)(lsn >> 32), (uint32)lsn, transactional, prefix,
+ sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c0b94bc..c21d2c7 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -17769,6 +17769,48 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-send-message-text">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-send-message-bytea">
+ <indexterm>
+ <primary>pg_logical_send_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written and decoded
+ immediately. The <parameter>prefix</parameter> has to be prefix which
+ was registered by a plugin. The <parameter>content</parameter> is
+ content of the message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index e841348..01a2e78 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,27 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..739ee10
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 88c3a49..110f958 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -115,6 +117,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -431,6 +437,37 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+ else if(!message->transactional &&
+ (SnapBuildCurrentState(ctx->snapshot_builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr, message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..998fb79 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn; /* beginning of commit record */
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional,
+ prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..2eb26d4 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function returning the changestream in binary, only peeking ahead.
+ */
+Datum
+pg_logical_send_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_send_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_send_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..e5f4feb
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,132 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/* List of registered logical message prefixes. */
+static List *LogicalMsgPrefixList = NIL;
+
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ ListCell *lc;
+ bool found = false;
+ xl_logical_message xlrec;
+
+ /* Check if the provided prefix is known to us. */
+ foreach(lc, LogicalMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("standby message prefix \"%s\" is not registered",
+ prefix)));
+
+ /*
+ * Force xid to be allocated if we're sending a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+void
+RegisterLogicalMsgPrefix(const char *prefix)
+{
+ ListCell *lc;
+ bool found = false;
+ MemoryContext oldcxt;
+
+ /* Check for duplicit registrations. */
+ foreach(lc, LogicalMsgPrefixList)
+ {
+ char *mp = lfirst(lc);
+
+ if (strcmp(prefix, mp) == 0)
+ {
+ found = true;
+ break;
+ }
+ }
+ if (found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("logical decoding message prefix \"%s\" is already registered",
+ prefix)));
+
+ oldcxt = MemoryContextSwitchTo(TopMemoryContext);
+ LogicalMsgPrefixList = lappend(LogicalMsgPrefixList, pstrdup(prefix));
+ MemoryContextSwitchTo(oldcxt);
+}
+
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 78acced..99e9ec7 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,11 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -599,6 +604,39 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz,
+ const char *msg)
+{
+ ReorderBufferTXN *txn = NULL;
+
+ if (transactional)
+ {
+ ReorderBufferChange *change;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ Assert(xid != InvalidTransactionId);
+ Assert(txn != NULL);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.transactional = true;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+ }
+ else
+ {
+ rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg);
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1465,6 +1503,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.transactional,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2117,6 +2163,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2354,6 +2415,18 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
data += len;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = pstrdup(data);
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, data + prefix_size,
+ message_size);
+
+ data += prefix_size + message_size;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 62b9125..76e41cd 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5084,6 +5084,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_text _null_ _null_ _null_ ));
+DESCR("send a textual message");
+DATA(insert OID = 3578 ( pg_logical_send_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_send_message_bytea _null_ _null_ _null_ ));
+DESCR("send a binary message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..cd3373e 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_send_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_send_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..e4223f4
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,42 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+extern void RegisterLogicalMsgPrefix(const char *prefix);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 2abee0a..f51f5c1 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -51,6 +51,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -95,6 +96,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ struct
+ {
+ bool transactional;
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -271,6 +280,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -297,6 +315,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -347,6 +366,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ bool transactional, const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
--
1.9.1
Hi,
I'm not really convinced by RegisterStandbyMsgPrefix() et al. There's
not much documentation about what it actually is supposed to
acomplish. Afaics you're basically forced to use
shared_preload_libraries with it right now? Also, iterating through a
linked list everytime something is logged doesn't seem very satisfying?
On 2016-02-24 18:35:16 +0100, Petr Jelinek wrote:
+SET synchronous_commit = on; +SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding'); + ?column? +---------- + init +(1 row)
+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1'); + ?column? +---------- + msg1 +(1 row)
Hm. Somehow 'sending' a message seems wrong here. Maybe 'emit'?
+ <row> + <entry id="pg-logical-send-message-text"> + <indexterm> + <primary>pg_logical_send_message</primary> + </indexterm> + <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal> + </entry> + <entry> + void + </entry> + <entry> + Write text logical decoding message. This can be used to pass generic + messages to logical decoding plugins through WAL. The parameter + <parameter>transactional</parameter> specifies if the message should + be part of current transaction or if it should be written and decoded + immediately. The <parameter>prefix</parameter> has to be prefix which + was registered by a plugin. The <parameter>content</parameter> is + content of the message. + </entry> + </row>
It's not decoded immediately, even if emitted non-transactionally.
+ <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + bool transactional, + const char *prefix, + Size message_size, + const char *message +);
We should at least document what txn is set to if not transactional.
+void +logicalmsg_desc(StringInfo buf, XLogReaderState *record) +{ + char *rec = XLogRecGetData(record); + xl_logical_message *xlrec = (xl_logical_message *) rec; + + appendStringInfo(buf, "%s message size %zu bytes", + xlrec->transactional ? "transactional" : "nontransactional", + xlrec->message_size); +}
Shouldn't we check
uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
if XLogRecGetInfo(record) == XLOG_LOGICAL_MESSAGE
here?
+const char * +logicalmsg_identify(uint8 info) +{ + return NULL; +}
Huh?
+void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + bool transactional, const char *prefix, Size msg_sz, + const char *msg) +{ + ReorderBufferTXN *txn = NULL; + + if (transactional) + { + ReorderBufferChange *change; + + txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true); + + Assert(xid != InvalidTransactionId); + Assert(txn != NULL); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.transactional = true; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); + } + else + { + rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg); + } +}
This approach prohibts catalog access when processing a nontransaction
message as there's no snapshot set up.
+ case REORDER_BUFFER_CHANGE_MESSAGE: + { + char *data; + size_t prefix_size = strlen(change->data.msg.prefix) + 1; + + sz += prefix_size + change->data.msg.message_size; + ReorderBufferSerializeReserve(rb, sz); + + data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange); + memcpy(data, change->data.msg.prefix, + prefix_size); + memcpy(data + prefix_size, change->data.msg.message, + change->data.msg.message_size); + break; + } case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT: { Snapshot snap; @@ -2354,6 +2415,18 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn, data += len; } break; + case REORDER_BUFFER_CHANGE_MESSAGE: + { + Size message_size = change->data.msg.message_size; + Size prefix_size = strlen(data) + 1; + + change->data.msg.prefix = pstrdup(data); + change->data.msg.message = palloc(message_size); + memcpy(change->data.msg.message, data + prefix_size, + message_size); + + data += prefix_size + message_size; + }
Please add a test exercising these paths.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hello,
On 27.02.2016 03:05, Andres Freund wrote:
Hi,
I'm not really convinced by RegisterStandbyMsgPrefix() et al. There's
not much documentation about what it actually is supposed to
acomplish. Afaics you're basically forced to use
shared_preload_libraries with it right now? Also, iterating through a
linked list everytime something is logged doesn't seem very satisfying?
I have did some tests with a simple plugin. I have used event triggers
to send messages. It works, but I agree with Andres. We have problems if
plugin is not loaded. For example, if you will execute the query:
SELECT 'msg2' FROM pg_logical_send_message(false, 'test', 'msg2');
you will get the error (if plugin which should register a prefix is not
loaded yet):
ERROR: standby message prefix "test" is not registered
Some stylistic note (logical.c):
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + bool transactional, const char *prefix, + Size sz, const char *message) +{ + LogicalDecodingContext *ctx = cache->private_data; + LogicalErrorCallbackState state; + ErrorContextCallback errcallback;
It should be written in the following way:
static void
message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr message_lsn,
bool transactional, const char *prefix,
Size sz, const char *message)
{
LogicalDecodingContext *ctx = cache->private_data;
LogicalErrorCallbackState state;
ErrorContextCallback errcallback;
--
Artur Zakirov
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
thanks for looking Andres,
On 27/02/16 01:05, Andres Freund wrote:
Hi,
I'm not really convinced by RegisterStandbyMsgPrefix() et al. There's
not much documentation about what it actually is supposed to
acomplish. Afaics you're basically forced to use
shared_preload_libraries with it right now? Also, iterating through a
linked list everytime something is logged doesn't seem very satisfying?
Well, my reasoning there was to stop multiple plugins from using same
prefix and for that you need some kind of registry. Making this a shared
catalog seemed like huge overkill given the potentially transient nature
of output plugins and this was the best I could come up with. And yes it
requires you to load your plugin before you can log a message for it.
I am not married to this solution so if you have better ideas or if you
think the clashes are not read issue, we can change it.
On 2016-02-24 18:35:16 +0100, Petr Jelinek wrote:
+SET synchronous_commit = on; +SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding'); + ?column? +---------- + init +(1 row)+SELECT 'msg1' FROM pg_logical_send_message(true, 'test', 'msg1'); + ?column? +---------- + msg1 +(1 row)Hm. Somehow 'sending' a message seems wrong here. Maybe 'emit'?
+ <row> + <entry id="pg-logical-send-message-text"> + <indexterm> + <primary>pg_logical_send_message</primary> + </indexterm> + <literal><function>pg_logical_send_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal> + </entry> + <entry> + void + </entry> + <entry> + Write text logical decoding message. This can be used to pass generic + messages to logical decoding plugins through WAL. The parameter + <parameter>transactional</parameter> specifies if the message should + be part of current transaction or if it should be written and decoded + immediately. The <parameter>prefix</parameter> has to be prefix which + was registered by a plugin. The <parameter>content</parameter> is + content of the message. + </entry> + </row>It's not decoded immediately, even if emitted non-transactionally.
Okay, immediately is somewhat misleading. How does "should be written
immediately and decoded as soon as logical decoding reads the WAL
record" sound ?
+ <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + bool transactional, + const char *prefix, + Size message_size, + const char *message +);We should at least document what txn is set to if not transactional.
Will do (it's NULL).
+void +logicalmsg_desc(StringInfo buf, XLogReaderState *record) +{ + char *rec = XLogRecGetData(record); + xl_logical_message *xlrec = (xl_logical_message *) rec; + + appendStringInfo(buf, "%s message size %zu bytes", + xlrec->transactional ? "transactional" : "nontransactional", + xlrec->message_size); +}Shouldn't we check
uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
if XLogRecGetInfo(record) == XLOG_LOGICAL_MESSAGE
here?
I thought it's kinda pointless, but we seem to be doing it in other
places so will add.
+void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + bool transactional, const char *prefix, Size msg_sz, + const char *msg) +{ + ReorderBufferTXN *txn = NULL; + + if (transactional) + { + ReorderBufferChange *change; + + txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true); + + Assert(xid != InvalidTransactionId); + Assert(txn != NULL); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.transactional = true; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); + } + else + { + rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg); + } +}This approach prohibts catalog access when processing a nontransaction
message as there's no snapshot set up.
Hmm I do see usefulness in having snapshot, although I wonder if that
does not kill the point of non-transactional messages. Question is then
though which snapshot should the message see, base_snapshot of
transaction? That would mean we'd have to call SnapBuildProcessChange
for non-transactional messages which we currently avoid. Alternatively
we could probably invent lighter version of that interface that would
just make sure builder->snapshot is valid and if not then build it but
I am honestly sure if that's a win or not.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
On 2016-02-28 22:44:12 +0100, Petr Jelinek wrote:
On 27/02/16 01:05, Andres Freund wrote:
I'm not really convinced by RegisterStandbyMsgPrefix() et al. There's
not much documentation about what it actually is supposed to
acomplish. Afaics you're basically forced to use
shared_preload_libraries with it right now? Also, iterating through a
linked list everytime something is logged doesn't seem very satisfying?Well, my reasoning there was to stop multiple plugins from using same prefix
and for that you need some kind of registry. Making this a shared catalog
seemed like huge overkill given the potentially transient nature of output
plugins and this was the best I could come up with. And yes it requires you
to load your plugin before you can log a message for it.
I think right now that's a solution that's worse than the problem. I'm
inclined to define the problem away with something like "The prefix
should be unique across different users of the messaging facility. Using
the extension name often is a good choice.".
+void +logicalmsg_desc(StringInfo buf, XLogReaderState *record) +{ + char *rec = XLogRecGetData(record); + xl_logical_message *xlrec = (xl_logical_message *) rec; + + appendStringInfo(buf, "%s message size %zu bytes", + xlrec->transactional ? "transactional" : "nontransactional", + xlrec->message_size); +}Shouldn't we check
uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
if XLogRecGetInfo(record) == XLOG_LOGICAL_MESSAGE
here?I thought it's kinda pointless, but we seem to be doing it in other places
so will add.
It leads to a segfault or something similar when adding further message
types, without a compiler warning. So it seems to be a good idea to be
slightly careful.
+void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + bool transactional, const char *prefix, Size msg_sz, + const char *msg) +{ + ReorderBufferTXN *txn = NULL; + + if (transactional) + { + ReorderBufferChange *change; + + txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true); + + Assert(xid != InvalidTransactionId); + Assert(txn != NULL); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.transactional = true; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); + } + else + { + rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg); + } +}This approach prohibts catalog access when processing a nontransaction
message as there's no snapshot set up.Hmm I do see usefulness in having snapshot, although I wonder if that does
not kill the point of non-transactional messages.
I don't see how it would? It'd obviously have to be the catalog/historic
snapshot a transaction would have had if it started in that moment in
the original WAL stream?
Question is then though which snapshot should the message see,
base_snapshot of transaction?
Well, there'll not be a transaction, but something like snapbuild.c's
->snapshot ought to do the trick.
That would mean we'd have to call SnapBuildProcessChange for
non-transactional messages which we currently avoid. Alternatively we
could probably invent lighter version of that interface that would
just make sure builder->snapshot is valid and if not then build it
I think the latter is probably the direction we should go in.
I am honestly sure if that's a win or not.
I think it'll be confusing (bug inducing) if there's no snapshot for
non-transactional messages but for transactional ones, and it'll
severely limit the usefulness of the interface.
Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
attached is the newest version of the patch.
I removed the registry, renamed the 'send' to 'emit', documented the
callback parameters properly. I also added the test to ddl.sql for the
serialization and deserialization (and of course found a bug there) and
in general fixed all the stuff Andres reported.
(see more inline)
On 28/02/16 22:55, Andres Freund wrote:
+void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + bool transactional, const char *prefix, Size msg_sz, + const char *msg) +{ + ReorderBufferTXN *txn = NULL; + + if (transactional) + { + ReorderBufferChange *change; + + txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true); + + Assert(xid != InvalidTransactionId); + Assert(txn != NULL); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.transactional = true; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); + } + else + { + rb->message(rb, txn, lsn, transactional, prefix, msg_sz, msg); + } +}This approach prohibts catalog access when processing a nontransaction
message as there's no snapshot set up.Hmm I do see usefulness in having snapshot, although I wonder if that does
not kill the point of non-transactional messages.I don't see how it would? It'd obviously have to be the catalog/historic
snapshot a transaction would have had if it started in that moment in
the original WAL stream?Question is then though which snapshot should the message see,
base_snapshot of transaction?Well, there'll not be a transaction, but something like snapbuild.c's
->snapshot ought to do the trick.
Ok I added interface which returns either existing snapshot or makes new
one, seems like the most reasonable thing to do to me.
That would mean we'd have to call SnapBuildProcessChange for
non-transactional messages which we currently avoid. Alternatively we
could probably invent lighter version of that interface that would
just make sure builder->snapshot is valid and if not then build itI think the latter is probably the direction we should go in.
I am honestly sure if that's a win or not.
I think it'll be confusing (bug inducing) if there's no snapshot for
non-transactional messages but for transactional ones, and it'll
severely limit the usefulness of the interface.
Nono, I meant I am not sure if special interface is a win over just
using SnapBuildProcessChange() in practice.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-02-29.patchapplication/x-patch; name=logical-messages-2016-02-29.patchDownload
From 2f037a757d9cec09f04457d82cdd1256b8255b78 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++++--
contrib/test_decoding/expected/messages.out | 56 ++++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 19 ++++++
doc/src/sgml/func.sgml | 45 +++++++++++++
doc/src/sgml/logicaldecoding.sgml | 37 +++++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 49 ++++++++++++++
src/backend/replication/logical/logical.c | 38 +++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++++
src/backend/replication/logical/message.c | 87 +++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 67 +++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++++
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++++++
src/include/replication/output_plugin.h | 13 ++++
src/include/replication/reorderbuffer.h | 21 ++++++
src/include/replication/snapbuild.h | 2 +
26 files changed, 608 insertions(+), 12 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index a362e69..8fdcfbc 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl rewrite toast permissions decoding_in_xact decoding_into_rel \
- binary prepared replorigin
+ binary prepared replorigin messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 57a1289..4d13ad4 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check that a large, spooled, upsert works
INSERT INTO tr_etoomuch (id, data)
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index e311c59..2f31784 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..6395ffd 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,10 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,16 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ transactional, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c0b94bc..1201f9f 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -17769,6 +17769,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index e841348..93ffcfb 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>ctx</parameter> and <parameter>txn</parameter> parameters
+ have the same contents as for the <function>begin_cb</function> with the
+ difference that <parameter>txn</parameter> is null for non-transactional
+ messages. The <parameter>lsn</parameter> has WAL position of the message.
+ The <parameter>transactional</parameter> says if he message was sent as
+ transactional or not. The <parameter>prefix</parameter> is arbitrary
+ textual prefix which can be used for identifying interesting messages
+ for the current plugin. And finally the <parameter>message</parameter>
+ parameter holds the actual message of <parameter>message_size</parameter>
+ size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..e5e84dd
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if (info & ~XLR_INFO_MASK == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 88c3a49..368a2f6 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -115,6 +117,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -431,6 +437,49 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional)
+ {
+ if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ }
+ else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT &&
+ !SnapBuildXactNeedsSkip(builder, buf->origptr))
+ {
+ volatile Snapshot snapshot_now;
+ ReorderBuffer *rb = ctx->reorder;
+
+ /* setup snapshot to allow catalog access */
+ snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r));
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ rb->message(rb, NULL, buf->origptr, false, message->message,
+ message->message_size,
+ message->message + message->prefix_size);
+ TeardownHistoricSnapshot(false);
+ }
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..3974cd4 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional,
+ prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..fe22daa 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function returning the changestream in binary, only peeking ahead.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..214be45
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,87 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 78acced..0312346 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -599,6 +607,26 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg)
+{
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.transactional = true;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1465,6 +1493,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.transactional,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2117,6 +2153,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2354,6 +2405,22 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
data += len;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ /* prefix includes trailing zero */
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ data += prefix_size;
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ message_size);
+ memcpy(change->data.msg.message, data, message_size);
+ data += message_size;
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index ed823ec..d7829c3 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 62b9125..ff722fe 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5084,6 +5084,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..b1730c9
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index d33ea27..a3af5a6 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -51,6 +51,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -95,6 +96,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ struct
+ {
+ bool transactional;
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -271,6 +280,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -297,6 +315,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -347,6 +366,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
I think here
+const char * +logicalmsg_identify(uint8 info) +{ + if (info & ~XLR_INFO_MASK == XLOG_LOGICAL_MESSAGE) + return "MESSAGE"; + + return NULL; +}
we should use brackets
const char *
logicalmsg_identify(uint8 info)
{
if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
return "MESSAGE";
return NULL;
}
Because of operator priorities
http://en.cppreference.com/w/c/language/operator_precedence we may get
errors.
On 01.03.2016 00:10, Petr Jelinek wrote:
Hi,
attached is the newest version of the patch.
I removed the registry, renamed the 'send' to 'emit', documented the
callback parameters properly. I also added the test to ddl.sql for the
serialization and deserialization (and of course found a bug there) and
in general fixed all the stuff Andres reported.(see more inline)
--
Artur Zakirov
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 08/03/16 21:21, Artur Zakirov wrote:
I think here
+const char * +logicalmsg_identify(uint8 info) +{ + if (info & ~XLR_INFO_MASK == XLOG_LOGICAL_MESSAGE) + return "MESSAGE"; + + return NULL; +}we should use brackets
const char *
logicalmsg_identify(uint8 info)
{
if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
return "MESSAGE";return NULL;
}
Correct, fixed, thanks.
I also rebased this as there was conflict after the fixes to logical
decoding by Andres.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-10.patchtext/x-patch; name=logical-messages-2016-03-10.patchDownload
From f65b4d858067af520e853cafb4fdfd11b6b3fdc0 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++++--
contrib/test_decoding/expected/messages.out | 56 ++++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 19 ++++++
doc/src/sgml/func.sgml | 45 +++++++++++++
doc/src/sgml/logicaldecoding.sgml | 37 +++++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 49 ++++++++++++++
src/backend/replication/logical/logical.c | 38 +++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++++
src/backend/replication/logical/message.c | 87 +++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 67 +++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++++
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++++++
src/include/replication/output_plugin.h | 13 ++++
src/include/replication/reorderbuffer.h | 21 ++++++
src/include/replication/snapbuild.h | 2 +
26 files changed, 608 insertions(+), 12 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 200c43e..7568531 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..6395ffd 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,10 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,16 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ transactional, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 4b5ee81..0f6bc20 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18125,6 +18125,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..f4e4506 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>ctx</parameter> and <parameter>txn</parameter> parameters
+ have the same contents as for the <function>begin_cb</function> with the
+ difference that <parameter>txn</parameter> is null for non-transactional
+ messages. The <parameter>lsn</parameter> has WAL position of the message.
+ The <parameter>transactional</parameter> says if he message was sent as
+ transactional or not. The <parameter>prefix</parameter> is arbitrary
+ textual prefix which can be used for identifying interesting messages
+ for the current plugin. And finally the <parameter>message</parameter>
+ parameter holds the actual message of <parameter>message_size</parameter>
+ size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..b945d35 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,49 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional)
+ {
+ if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ }
+ else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT &&
+ !SnapBuildXactNeedsSkip(builder, buf->origptr))
+ {
+ volatile Snapshot snapshot_now;
+ ReorderBuffer *rb = ctx->reorder;
+
+ /* setup snapshot to allow catalog access */
+ snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r));
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ rb->message(rb, NULL, buf->origptr, false, message->message,
+ message->message_size,
+ message->message + message->prefix_size);
+ TeardownHistoricSnapshot(false);
+ }
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..7a34251 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size sz, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional,
+ prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..fe22daa 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function returning the changestream in binary, only peeking ahead.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..214be45
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,87 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..bb6f438 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,26 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg)
+{
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.transactional = true;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1521,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.transactional,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2195,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2466,22 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ /* prefix includes trailing zero */
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ data += prefix_size;
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ message_size);
+ memcpy(change->data.msg.message, data, message_size);
+ data += message_size;
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a0f821a..bae5540 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5095,6 +5095,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f f f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..b1730c9
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..c7a970a 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ struct
+ {
+ bool transactional;
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +318,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +369,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
On 3/9/16 6:58 PM, Petr Jelinek wrote:
On 08/03/16 21:21, Artur Zakirov wrote:
I think here
+const char * +logicalmsg_identify(uint8 info) +{ + if (info & ~XLR_INFO_MASK == XLOG_LOGICAL_MESSAGE) + return "MESSAGE"; + + return NULL; +}we should use brackets
const char *
logicalmsg_identify(uint8 info)
{
if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
return "MESSAGE";return NULL;
}Correct, fixed, thanks.
I also rebased this as there was conflict after the fixes to logical
decoding by Andres.
This patch applies cleanly and is ready for review with no outstanding
issues that I can see. Simon and Artur, you are both signed up as
reviewers. Care to take a crack at it?
Thanks,
--
-David
david@pgmasters.net
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 16.03.2016 18:56, David Steele wrote:
This patch applies cleanly and is ready for review with no outstanding
issues that I can see. Simon and Artur, you are both signed up as
reviewers. Care to take a crack at it?Thanks,
I have tested the patch once again and have looked the code. It looks
good for me. I haven't any observation.
The patch does its function correctly. I have tested it with a plugin,
which writes DDL queries to the WAL using a hook and replicates this
queries at subscribers.
If Simon is not against, the patch can be marked as "Ready for Commiter".
--
Artur Zakirov
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
a few comments about the last version of the patch:
1) LogicalDecodeMessageCB
Do we actually need the 'transactional' parameter here? I mean, having
the 'txn' should be enough, as
transactional = (txt != NULL)
Of course, having a simple flag is more convenient.
2) pg_logical_emit_message_bytea / pg_logical_emit_message_text
The comment before _bytea is wrong - it's just a copy'n'paste from the
preceding function (pg_logical_slot_peek_binary_changes). _text has no
comment at all, but it's true it's just a simple _bytea wrapper.
The main issue here however is that the functions are not defined as
strict, but ignore the possibility that the parameters might be NULL. So
for example this crashes the backend
SELECT pg_logical_emit_message(NULL::boolean, NULL::text, NULL::text);
3) ReorderBufferQueueMessage
No comment. Not a big deal I guess, the method is simple enough, but why
to break the rule when all the other methods around have at least a
short one?
4) ReorderBufferChange
The new struct in the 'union' would probably deserve at least a short
comment explaining the purpose (just like the other structs around).
regards
--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 17 March 2016 at 19:08, Artur Zakirov <a.zakirov@postgrespro.ru> wrote:
On 16.03.2016 18:56, David Steele wrote:
This patch applies cleanly and is ready for review with no outstanding
issues that I can see. Simon and Artur, you are both signed up as
reviewers. Care to take a crack at it?Thanks,
I have tested the patch once again and have looked the code. It looks good
for me. I haven't any observation.The patch does its function correctly. I have tested it with a plugin,
which writes DDL queries to the WAL using a hook and replicates this
queries at subscribers.
Would you mind sharing the plugin here? I could add it to src/test/modules
and add some t/ tests so it runs under the TAP test framework.
--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
On 17.03.2016 15:42, Craig Ringer wrote:
Would you mind sharing the plugin here? I could add it to
src/test/modules and add some t/ tests so it runs under the TAP test
framework.--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Of course. I attached it.
In a provider node you need to set shared_preload_libraries to
'pg_ddl_decode'.
In a subscriber node you need:
1 - install pg_ddl_decode by the command:
CREATE EXTENSION pg_ddl_decode;
2 - call the function:
SELECT ddl_create_slot('host=providerhost port=5432 dbname=db');
3 - after some DDL queries in provider you need to execute in a
subscriber the command manually:
SELECT ddl_get_changes('host=providerhost port=5432 dbname=db');
I warn that this plugin has ugly code sometimes. And ddl_get_changes()
has a infinite loop. But it shows concept of DDL replication.
Also this plugin uses some functions from pglogical_output and pglogical.
Earlier I have did tests with pglogical. I attached the patch for pglogical.
--
Artur Zakirov
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
Attachments:
pglogical-ddl.patchtext/x-patch; name=pglogical-ddl.patchDownload
diff --git a/contrib/pglogical/pglogical.c b/contrib/pglogical/pglogical.c
index f25b91d..6935d0b 100644
--- a/contrib/pglogical/pglogical.c
+++ b/contrib/pglogical/pglogical.c
@@ -25,9 +25,13 @@
#include "mb/pg_wchar.h"
+#include "replication/message.h"
+
#include "storage/ipc.h"
#include "storage/proc.h"
+#include "tcop/utility.h"
+
#include "utils/builtins.h"
#include "utils/lsyscache.h"
@@ -51,8 +55,10 @@ bool pglogical_synchronous_commit = false;
char *pglogical_temp_directory;
void _PG_init(void);
+void _PG_fini(void);
void pglogical_supervisor_main(Datum main_arg);
+static ProcessUtility_hook_type prev_utility_hook = NULL;
/*
* Ensure string is not longer than maxlen.
@@ -400,6 +406,38 @@ pglogical_supervisor_main(Datum main_arg)
proc_exit(1);
}
+static void
+emit_message_utility(Node *parsetree,
+ const char *queryString,
+ ProcessUtilityContext context,
+ ParamListInfo params,
+ DestReceiver *dest,
+ char *completionTag)
+{
+ NodeTag stmtType = nodeTag(parsetree);
+
+ if (context == PROCESS_UTILITY_TOPLEVEL &&
+ /*
+ * Create and drop statements are handled. Can be added more tags from
+ * nodes.h.
+ */
+ (stmtType == T_CreateStmt || stmtType == T_DropStmt))
+ LogLogicalMessage("pglogical",
+ queryString, strlen(queryString), true);
+
+ /*
+ * Fallback to normal process, be it the previous hook loaded
+ * or the in-core code path if the previous hook does not exist.
+ */
+ if (prev_utility_hook)
+ (*prev_utility_hook) (parsetree, queryString,
+ context, params,
+ dest, completionTag);
+ else
+ standard_ProcessUtility(parsetree, queryString,
+ context, params,
+ dest, completionTag);
+}
/*
* Entry point for this module.
@@ -442,6 +480,10 @@ _PG_init(void)
"/tmp", PGC_SIGHUP,
0,
NULL, NULL, NULL);
+
+ prev_utility_hook = ProcessUtility_hook;
+ ProcessUtility_hook = emit_message_utility;
+
if (IsBinaryUpgrade)
return;
@@ -465,3 +507,9 @@ _PG_init(void)
RegisterBackgroundWorker(&bgw);
}
+
+void
+_PG_fini(void)
+{
+ ProcessUtility_hook = prev_utility_hook;
+}
\ No newline at end of file
diff --git a/contrib/pglogical/pglogical_apply.c b/contrib/pglogical/pglogical_apply.c
index bf5615c..8f78fef 100644
--- a/contrib/pglogical/pglogical_apply.c
+++ b/contrib/pglogical/pglogical_apply.c
@@ -684,6 +684,19 @@ handle_delete(StringInfo s)
CommandCounterIncrement();
}
+static void
+handle_ddl(StringInfo s)
+{
+ char *query = pglogical_read_ddl(s);
+ bool started_tx = ensure_transaction();
+
+ /* Execute the query locally. */
+ pglogical_execute_sql_command(query, GetUserNameFromId(GetUserId(), false),
+ started_tx);
+
+ CommandCounterIncrement();
+}
+
inline static bool
getmsgisend(StringInfo msg)
{
@@ -1045,6 +1058,10 @@ replication_handler(StringInfo s)
case 'S':
handle_startup(s);
break;
+ /* DDL MESSAGE */
+ case 'M':
+ handle_ddl(s);
+ break;
default:
elog(ERROR, "unknown action of type %c", action);
}
diff --git a/contrib/pglogical/pglogical_proto.c b/contrib/pglogical/pglogical_proto.c
index 558a955..133c1c0 100644
--- a/contrib/pglogical/pglogical_proto.c
+++ b/contrib/pglogical/pglogical_proto.c
@@ -230,6 +230,27 @@ pglogical_read_delete(StringInfo in, LOCKMODE lockmode,
return rel;
}
+/*
+ * Read DDL command from stream.
+ *
+ * Returns DDL query.
+ */
+char *
+pglogical_read_ddl(StringInfo in)
+{
+ uint8 flags;
+ Size sz;
+
+ /* read the flags */
+ flags = pq_getmsgbyte(in);
+ Assert(flags == 0);
+
+ /* read the message lengh */
+ sz = pq_getmsgint(in, 4);
+
+ /* read the message */
+ return (char *) pq_getmsgbytes(in, sz);
+}
/*
* Read tuple in remote format from stream.
@@ -407,7 +428,7 @@ pglogical_read_attrs(StringInfo in, char ***attrnames, int *nattrnames)
if (blocktype != 'C')
elog(ERROR, "expected COLUMN, got %c", blocktype);
flags = pq_getmsgbyte(in);
-// Assert(flags == 0);
+ Assert(flags >= 0);
blocktype = pq_getmsgbyte(in); /* column name block follows */
if (blocktype != 'N')
diff --git a/contrib/pglogical/pglogical_proto.h b/contrib/pglogical/pglogical_proto.h
index 3229c322..1d2e055 100644
--- a/contrib/pglogical/pglogical_proto.h
+++ b/contrib/pglogical/pglogical_proto.h
@@ -38,5 +38,6 @@ extern PGLogicalRelation *pglogical_read_update(StringInfo in, LOCKMODE lockmode
PGLogicalTupleData *oldtup, PGLogicalTupleData *newtup);
extern PGLogicalRelation *pglogical_read_delete(StringInfo in, LOCKMODE lockmode,
PGLogicalTupleData *oldtup);
+extern char *pglogical_read_ddl(StringInfo in);
#endif /* PGLOGICAL_PROTO_H */
diff --git a/contrib/pglogical/pglogical_worker.c b/contrib/pglogical/pglogical_worker.c
index cc37866..1a3ca21 100644
--- a/contrib/pglogical/pglogical_worker.c
+++ b/contrib/pglogical/pglogical_worker.c
@@ -406,7 +406,11 @@ pglogical_worker_shmem_startup(void)
if (!found)
{
- PGLogicalCtx->lock = LWLockAssign();
+ #if PG_VERSION_NUM >= 90600
+ PGLogicalCtx->lock = &(GetNamedLWLockTranche(EXTENSION_NAME))->lock;
+ #else
+ PGLogicalCtx->lock = LWLockAssign();
+ #endif
PGLogicalCtx->supervisor = NULL;
PGLogicalCtx->total_workers = max_worker_processes;
memset(PGLogicalCtx->workers, 0,
@@ -430,7 +434,11 @@ pglogical_worker_shmem_init(void)
* tries to allocate or free blocks from this array at once. There won't
* be enough contention to make anything fancier worth doing.
*/
+ #if PG_VERSION_NUM >= 90600
+ RequestNamedLWLockTranche(EXTENSION_NAME, 1);
+ #else
RequestAddinLWLocks(1);
+ #endif
/*
* Whether this is a first startup or crash recovery, we'll be re-initing
diff --git a/contrib/pglogical_output/pglogical_output.c b/contrib/pglogical_output/pglogical_output.c
index 5262d82..3181c6c 100644
--- a/contrib/pglogical_output/pglogical_output.c
+++ b/contrib/pglogical_output/pglogical_output.c
@@ -66,6 +66,10 @@ static void pg_decode_commit_txn(LogicalDecodingContext *ctx,
static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferTXN *txn, Relation rel,
ReorderBufferChange *change);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
#ifdef HAVE_REPLICATION_ORIGINS
static bool pg_decode_origin_filter(LogicalDecodingContext *ctx,
@@ -87,6 +91,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->begin_cb = pg_decode_begin_txn;
cb->change_cb = pg_decode_change;
cb->commit_cb = pg_decode_commit_txn;
+ cb->message_cb = pg_decode_message;
#ifdef HAVE_REPLICATION_ORIGINS
cb->filter_by_origin_cb = pg_decode_origin_filter;
#endif
@@ -254,7 +259,7 @@ pg_decode_startup(LogicalDecodingContext * ctx, OutputPluginOptions *opt,
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("client requested protocol %s but only \"json\" or \"native\" are supported",
- data->client_protocol_format)));
+ data->client_protocol_format)));
}
/* check for encoding match if specific encoding demanded by client */
@@ -512,6 +517,22 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
MemoryContextReset(data->context);
}
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message)
+{
+ PGLogicalOutputData *data = ctx->output_plugin_private;
+
+ if (strcmp(prefix, "pglogical") == 0)
+ {
+ OutputPluginPrepareWrite(ctx, true);
+ data->api->write_ddl(ctx->out, data, message, sz);
+ OutputPluginWrite(ctx, true);
+ }
+}
+
#ifdef HAVE_REPLICATION_ORIGINS
/*
* Decide if the whole transaction with specific origin should be filtered out.
diff --git a/contrib/pglogical_output/pglogical_proto.c b/contrib/pglogical_output/pglogical_proto.c
index 47a883f..a1e6b85 100644
--- a/contrib/pglogical_output/pglogical_proto.c
+++ b/contrib/pglogical_output/pglogical_proto.c
@@ -31,6 +31,7 @@ pglogical_init_api(PGLogicalProtoType typ)
res->write_insert = pglogical_json_write_insert;
res->write_update = pglogical_json_write_update;
res->write_delete = pglogical_json_write_delete;
+ res->write_ddl = pglogical_json_write_ddl;
res->write_startup_message = json_write_startup_message;
}
else
@@ -42,6 +43,7 @@ pglogical_init_api(PGLogicalProtoType typ)
res->write_insert = pglogical_write_insert;
res->write_update = pglogical_write_update;
res->write_delete = pglogical_write_delete;
+ res->write_ddl = pglogical_write_ddl;
res->write_startup_message = write_startup_message;
}
diff --git a/contrib/pglogical_output/pglogical_proto.h b/contrib/pglogical_output/pglogical_proto.h
index 27897e2..032803b 100644
--- a/contrib/pglogical_output/pglogical_proto.h
+++ b/contrib/pglogical_output/pglogical_proto.h
@@ -34,6 +34,8 @@ typedef void (*pglogical_write_update_fn)(StringInfo out, struct PGLogicalOutput
HeapTuple newtuple);
typedef void (*pglogical_write_delete_fn)(StringInfo out, struct PGLogicalOutputData *data,
Relation rel, HeapTuple oldtuple);
+typedef void (*pglogical_write_ddl_fn)(StringInfo out, struct PGLogicalOutputData *data,
+ const char *message, Size sz);
typedef void (*write_startup_message_fn)(StringInfo out, List *msg);
@@ -46,6 +48,7 @@ typedef struct PGLogicalProtoAPI
pglogical_write_insert_fn write_insert;
pglogical_write_update_fn write_update;
pglogical_write_delete_fn write_delete;
+ pglogical_write_ddl_fn write_ddl;
write_startup_message_fn write_startup_message;
} PGLogicalProtoAPI;
diff --git a/contrib/pglogical_output/pglogical_proto_json.c b/contrib/pglogical_output/pglogical_proto_json.c
index ae5a591..70e23c5 100644
--- a/contrib/pglogical_output/pglogical_proto_json.c
+++ b/contrib/pglogical_output/pglogical_proto_json.c
@@ -178,6 +178,18 @@ pglogical_json_write_delete(StringInfo out, PGLogicalOutputData *data,
}
/*
+ * Write DDL logical message to the output stream.
+ */
+void
+pglogical_json_write_ddl(StringInfo out, PGLogicalOutputData *data,
+ const char *message, Size sz)
+{
+ appendStringInfoString(out, "{\"action\":\"M\", \"message\":");
+ appendBinaryStringInfo(out, message, sz);
+ appendStringInfoString(out, "}");
+}
+
+/*
* The startup message should be constructed as a json object, one
* key/value per DefElem list member.
*/
diff --git a/contrib/pglogical_output/pglogical_proto_json.h b/contrib/pglogical_output/pglogical_proto_json.h
index d853e9e..03ccfc0 100644
--- a/contrib/pglogical_output/pglogical_proto_json.h
+++ b/contrib/pglogical_output/pglogical_proto_json.h
@@ -26,6 +26,8 @@ extern void pglogical_json_write_update(StringInfo out, PGLogicalOutputData *dat
HeapTuple newtuple);
extern void pglogical_json_write_delete(StringInfo out, PGLogicalOutputData *data,
Relation rel, HeapTuple oldtuple);
+extern void pglogical_json_write_ddl(StringInfo out, PGLogicalOutputData *data,
+ const char *message, Size sz);
extern void json_write_startup_message(StringInfo out, List *msg);
diff --git a/contrib/pglogical_output/pglogical_proto_native.c b/contrib/pglogical_output/pglogical_proto_native.c
index 87ec453..f6630a2 100644
--- a/contrib/pglogical_output/pglogical_proto_native.c
+++ b/contrib/pglogical_output/pglogical_proto_native.c
@@ -318,6 +318,24 @@ pglogical_write_delete(StringInfo out, PGLogicalOutputData *data,
}
/*
+ * Write DDL logical message to the output stream.
+ */
+void
+pglogical_write_ddl(StringInfo out, PGLogicalOutputData *data,
+ const char *message, Size sz)
+{
+ uint8 flags = 0;
+
+ pq_sendbyte(out, 'M'); /* action DDL message */
+
+ /* send the flags field */
+ pq_sendbyte(out, flags);
+
+ pq_sendint(out, sz, 4); /* message length */
+ pq_sendbytes(out, message, sz);
+}
+
+/*
* Most of the brains for startup message creation lives in
* pglogical_config.c, so this presently just sends the set of key/value pairs.
*/
diff --git a/contrib/pglogical_output/pglogical_proto_native.h b/contrib/pglogical_output/pglogical_proto_native.h
index d3ba125..c02d63e 100644
--- a/contrib/pglogical_output/pglogical_proto_native.h
+++ b/contrib/pglogical_output/pglogical_proto_native.h
@@ -41,6 +41,8 @@ extern void pglogical_write_update(StringInfo out, PGLogicalOutputData *data,
HeapTuple newtuple);
extern void pglogical_write_delete(StringInfo out, PGLogicalOutputData *data,
Relation rel, HeapTuple oldtuple);
+extern void pglogical_write_ddl(StringInfo out, PGLogicalOutputData *data,
+ const char *message, Size sz);
extern void write_startup_message(StringInfo out, List *msg);
Hi,
thanks for review.
On 17/03/16 13:36, Tomas Vondra wrote:
Hi,
a few comments about the last version of the patch:
1) LogicalDecodeMessageCB
Do we actually need the 'transactional' parameter here? I mean, having
the 'txn' should be enough, astransactional = (txt != NULL)
Agreed. Same goes for the ReoderBufferChange struct btw, only
transactional messages go there so no point in marking them as such.
2) pg_logical_emit_message_bytea / pg_logical_emit_message_text
The comment before _bytea is wrong - it's just a copy'n'paste from the
preceding function (pg_logical_slot_peek_binary_changes). _text has no
comment at all, but it's true it's just a simple _bytea wrapper.
Heh, blind.
The main issue here however is that the functions are not defined as
strict, but ignore the possibility that the parameters might be NULL. So
for example this crashes the backendSELECT pg_logical_emit_message(NULL::boolean, NULL::text, NULL::text);
Good point.
3) ReorderBufferQueueMessage
No comment. Not a big deal I guess, the method is simple enough, but why
to break the rule when all the other methods around have at least a
short one?
Yeah I sometimes am not sure if there is a point to put comment to tiny
straightforward functions that do more or less same as the one above.
But it's public API so probably better to have one.
4) ReorderBufferChange
The new struct in the 'union' would probably deserve at least a short
comment explaining the purpose (just like the other structs around).
Okay.
Updated version attached.
(BTW please try to CC author of the patch when reviewing)
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-21.patchtext/x-diff; name=logical-messages-2016-03-21.patchDownload
From 4515b5b3e8f5ddb96124e859b968c43c27f79be3 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++++--
contrib/test_decoding/expected/messages.out | 56 ++++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 17 +++++
doc/src/sgml/func.sgml | 45 +++++++++++++
doc/src/sgml/logicaldecoding.sgml | 34 ++++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 49 ++++++++++++++
src/backend/replication/logical/logical.c | 37 +++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++++
src/backend/replication/logical/message.c | 87 +++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 68 +++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++++
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++++++
src/include/replication/output_plugin.h | 12 ++++
src/include/replication/reorderbuffer.h | 20 ++++++
src/include/replication/snapbuild.h | 2 +
26 files changed, 601 insertions(+), 12 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..1997774 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,9 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ const char *prefix, Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +86,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +476,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ txn != NULL, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index ae93e69..02f556b 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18238,6 +18238,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..7c9d03a 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,39 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>ctx</parameter> and <parameter>txn</parameter> parameters
+ have the same contents as for the <function>begin_cb</function> with the
+ difference that <parameter>txn</parameter> is null for non-transactional
+ messages. The <parameter>lsn</parameter> has WAL position of the message.
+ The <parameter>prefix</parameter> is arbitrary textual prefix which can
+ be used for identifying interesting messages for the current plugin.
+ And finally the <parameter>message</parameter> parameter holds the
+ actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..98ec5d9 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,49 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional)
+ {
+ if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ }
+ else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT &&
+ !SnapBuildXactNeedsSkip(builder, buf->origptr))
+ {
+ volatile Snapshot snapshot_now;
+ ReorderBuffer *rb = ctx->reorder;
+
+ /* setup snapshot to allow catalog access */
+ snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r));
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ rb->message(rb, NULL, buf->origptr, message->message,
+ message->message_size,
+ message->message + message->prefix_size);
+ TeardownHistoricSnapshot(false);
+ }
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..7863d52 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,39 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..552dac3 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..214be45
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,87 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..d6d436e 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,28 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg)
+{
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1523,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2196,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2467,22 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ /* prefix includes trailing zero */
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ data += prefix_size;
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ message_size);
+ memcpy(change->data.msg.message, data, message_size);
+ data += message_size;
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..3713739 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5114,6 +5114,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..b1730c9
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..fb5b758 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,17 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +107,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..c12dbf4 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,14 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +317,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +368,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
On 18 March 2016 at 20:36, Artur Zakirov <a.zakirov@postgrespro.ru> wrote:
On 17.03.2016 15:42, Craig Ringer wrote:
Would you mind sharing the plugin here? I could add it to
src/test/modules and add some t/ tests so it runs under the TAP test
framework.--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & ServicesOf course. I attached it.
Thanks for that.
Since it incorporates fairly significant chunks of code from a number of
places and is really a proof of concept rather than demo/example I don't
think it can really be included as a TAP test module, not without more
rewriting than I currently have time for anyway.
But it shows concept of DDL replication.
Yeah, a part of it anyway. As I outlined in another thread some time ago,
getting DDL replication right is fairly tricky and requires more changes
than just capturing the raw DDL (or even deparsed DDL) and sending it over
the wire.
It's a useful proof of concept, but way too big to use as an in-tree
regression for logical WAL messages as I was hoping.
--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Just noticed there is missing symlink in the pg_xlogdump.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-21-2.patchtext/x-diff; name=logical-messages-2016-03-21-2.patchDownload
From f47730e5e8ef5797c7595aafcbf8cff3b375d0ad Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++++--
contrib/test_decoding/expected/messages.out | 56 ++++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 17 +++++
doc/src/sgml/func.sgml | 45 +++++++++++++
doc/src/sgml/logicaldecoding.sgml | 34 ++++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 49 ++++++++++++++
src/backend/replication/logical/logical.c | 37 +++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++++
src/backend/replication/logical/message.c | 87 +++++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 68 +++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++++
src/bin/pg_xlogdump/logicalmsgdesc.c | 1 +
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++++++
src/include/replication/output_plugin.h | 12 ++++
src/include/replication/reorderbuffer.h | 20 ++++++
src/include/replication/snapbuild.h | 2 +
27 files changed, 602 insertions(+), 12 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 120000 src/bin/pg_xlogdump/logicalmsgdesc.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..1997774 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,9 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ const char *prefix, Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +86,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +476,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ txn != NULL, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index ae93e69..02f556b 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18238,6 +18238,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Write binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..7c9d03a 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,39 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>ctx</parameter> and <parameter>txn</parameter> parameters
+ have the same contents as for the <function>begin_cb</function> with the
+ difference that <parameter>txn</parameter> is null for non-transactional
+ messages. The <parameter>lsn</parameter> has WAL position of the message.
+ The <parameter>prefix</parameter> is arbitrary textual prefix which can
+ be used for identifying interesting messages for the current plugin.
+ And finally the <parameter>message</parameter> parameter holds the
+ actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..98ec5d9 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,49 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional)
+ {
+ if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ }
+ else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT &&
+ !SnapBuildXactNeedsSkip(builder, buf->origptr))
+ {
+ volatile Snapshot snapshot_now;
+ ReorderBuffer *rb = ctx->reorder;
+
+ /* setup snapshot to allow catalog access */
+ snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r));
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ rb->message(rb, NULL, buf->origptr, message->message,
+ message->message_size,
+ message->message + message->prefix_size);
+ TeardownHistoricSnapshot(false);
+ }
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..7863d52 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,39 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..552dac3 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..214be45
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,87 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The prefix has to be registered before the message using that
+ * prefix can be written to XLOG. The prefix can be registered exactly once to
+ * avoid situation where multiple third party extensions try to use same
+ * prefix.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..d6d436e 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,28 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg)
+{
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1523,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2196,21 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ size_t prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size;
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ memcpy(data + prefix_size, change->data.msg.message,
+ change->data.msg.message_size);
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2467,22 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size message_size = change->data.msg.message_size;
+ /* prefix includes trailing zero */
+ Size prefix_size = strlen(data) + 1;
+
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ data += prefix_size;
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ message_size);
+ memcpy(change->data.msg.message, data, message_size);
+ data += message_size;
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/logicalmsgdesc.c b/src/bin/pg_xlogdump/logicalmsgdesc.c
new file mode 120000
index 0000000..541fd5e
--- /dev/null
+++ b/src/bin/pg_xlogdump/logicalmsgdesc.c
@@ -0,0 +1 @@
+../../../src/backend/access/rmgrdesc/logicalmsgdesc.c
\ No newline at end of file
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..3713739 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5114,6 +5114,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..b1730c9
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefx of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..fb5b758 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,17 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +107,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..c12dbf4 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ size_t message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,14 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +317,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +368,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
On 2016-03-21 18:10:55 +0100, Petr Jelinek wrote:
Just noticed there is missing symlink in the pg_xlogdump.
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 120000 src/bin/pg_xlogdump/logicalmsgdesc.c
Uh, src/bin/pg_xlogdump/logicalmsgdesc.c shouldn't be there. The symlink
is supposed to be automatically created by the Makefile.
Were you perhaps confused because it showed up in git status? If so,
that's probably because it isn't in
src/bin/pg_xlogdump/.gitignore. Perhaps we should change that file to
ignore *desc.c?
+ <row> + <entry id="pg-logical-emit-message-text"> + <indexterm> + <primary>pg_logical_emit_message</primary> + </indexterm> + <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal> + </entry> + <entry> + void + </entry> + <entry> + Write text logical decoding message. This can be used to pass generic + messages to logical decoding plugins through WAL. The parameter + <parameter>transactional</parameter> specifies if the message should + be part of current transaction or if it should be written immediately + and decoded as soon as the logical decoding reads the record. The + <parameter>prefix</parameter> is textual prefix used by the logical + decoding plugins to easily recognize interesting messages for them. + The <parameter>content</parameter> is the text of the message. + </entry> + </row>
s/write/emit/?
+ + <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + const char *prefix, + Size message_size, + const char *message +);
I see you removed the transactional parameter. I'm doubtful that that's
a good idea: It seems like it'd be rather helpful to pass the
transaction for a nontransaction message that's emitted while an xid was
assigned?
+/* + * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer(). + */ +static void +DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf) +{ + SnapBuild *builder = ctx->snapshot_builder; + XLogReaderState *r = buf->record; + uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK; + xl_logical_message *message; + + if (info != XLOG_LOGICAL_MESSAGE) + elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info); + + message = (xl_logical_message *) XLogRecGetData(r); + + if (message->transactional) + { + if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr)) + return; + + ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r), + buf->endptr, + message->message, /* first part of message is prefix */ + message->message_size, + message->message + message->prefix_size); + } + else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT && + !SnapBuildXactNeedsSkip(builder, buf->origptr)) + { + volatile Snapshot snapshot_now; + ReorderBuffer *rb = ctx->reorder; + + /* setup snapshot to allow catalog access */ + snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r)); + SetupHistoricSnapshot(snapshot_now, NULL); + rb->message(rb, NULL, buf->origptr, message->message, + message->message_size, + message->message + message->prefix_size); + TeardownHistoricSnapshot(false); + } +}
A number of things:
1) The SnapBuildProcessChange needs to be toplevel, not just for
transactional messages - we can't yet necessarily build a snapshot.
2) I'm inclined to move even the non-transactional stuff to reorderbuffer.
3) This lacks error handling, we surely don't want to error out while
still having the historic snapshot setup
4) Without 3) the volatile is bogus.
5) Misses a ReorderBufferProcessXid() call.
+ * Every message carries prefix to avoid conflicts between different decoding + * plugins. The prefix has to be registered before the message using that + * prefix can be written to XLOG. The prefix can be registered exactly once to + * avoid situation where multiple third party extensions try to use same + * prefix.
Outdated afaics?
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change) change->data.tp.oldtuple = NULL; } break; + case REORDER_BUFFER_CHANGE_MESSAGE: + if (change->data.msg.prefix != NULL) + pfree(change->data.msg.prefix); + change->data.msg.prefix = NULL; + if (change->data.msg.message != NULL) + pfree(change->data.msg.message); + change->data.msg.message = NULL; + break;
Hm, this will have some overhead, but I guess the messages won't be
super frequent, and usually not very large.
+/* + * Queue message into a transaction so it can be processed upon commit. + */ +void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + const char *prefix, Size msg_sz, const char *msg) +{ + ReorderBufferChange *change; + + Assert(xid != InvalidTransactionId); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); +}
I'm not sure right now if there's any guarantee that the current memory
context is meaningful here? IIRC other long-lived allocations explicitly
use a context?
+ case REORDER_BUFFER_CHANGE_MESSAGE: + { + char *data; + size_t prefix_size = strlen(change->data.msg.prefix) + 1; + + sz += prefix_size + change->data.msg.message_size; + ReorderBufferSerializeReserve(rb, sz); + + data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange); + memcpy(data, change->data.msg.prefix, + prefix_size); + memcpy(data + prefix_size, change->data.msg.message, + change->data.msg.message_size); + break; + }
Can you please include the sizes of the blocks explicitly, rather than
relying on 0 termination?
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL) PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL) PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL) +PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
Did you consider doing this via the standby rmgr instead?
+typedef struct xl_logical_message +{ + bool transactional; /* is message transactional? */ + size_t prefix_size; /* length of prefix */ + size_t message_size; /* size of the message */ + char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null + * terminated prefx of length + * prefix_size */ +} xl_logical_message;
"prefx".
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 22/03/16 12:47, Andres Freund wrote:
On 2016-03-21 18:10:55 +0100, Petr Jelinek wrote:
+ + <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + const char *prefix, + Size message_size, + const char *message +);I see you removed the transactional parameter. I'm doubtful that that's
a good idea: It seems like it'd be rather helpful to pass the
transaction for a nontransaction message that's emitted while an xid was
assigned?
Hmm but won't that give the output plugin even transactions that were
later aborted? That seems quite different behavior from how the txn
parameter works everywhere else.
+/* + * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer(). + */ +static void +DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf) +{ + SnapBuild *builder = ctx->snapshot_builder; + XLogReaderState *r = buf->record; + uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK; + xl_logical_message *message; + + if (info != XLOG_LOGICAL_MESSAGE) + elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info); + + message = (xl_logical_message *) XLogRecGetData(r); + + if (message->transactional) + { + if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr)) + return; + + ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r), + buf->endptr, + message->message, /* first part of message is prefix */ + message->message_size, + message->message + message->prefix_size); + } + else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT && + !SnapBuildXactNeedsSkip(builder, buf->origptr)) + { + volatile Snapshot snapshot_now; + ReorderBuffer *rb = ctx->reorder; + + /* setup snapshot to allow catalog access */ + snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r)); + SetupHistoricSnapshot(snapshot_now, NULL); + rb->message(rb, NULL, buf->origptr, message->message, + message->message_size, + message->message + message->prefix_size); + TeardownHistoricSnapshot(false); + } +}A number of things:
1) The SnapBuildProcessChange needs to be toplevel, not just for
transactional messages - we can't yet necessarily build a snapshot.
Nope, the snapshot state is checked in the else if.
2) I'm inclined to move even the non-transactional stuff to reorderbuffer.
Well, it's not doing anything with reorderbuffer but sure it can be done
(didn't do it in the attached though).
3) This lacks error handling, we surely don't want to error out while
still having the historic snapshot setup
4) Without 3) the volatile is bogus.
5) Misses a ReorderBufferProcessXid() call.
Fixed (all 3 above).
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change) change->data.tp.oldtuple = NULL; } break; + case REORDER_BUFFER_CHANGE_MESSAGE: + if (change->data.msg.prefix != NULL) + pfree(change->data.msg.prefix); + change->data.msg.prefix = NULL; + if (change->data.msg.message != NULL) + pfree(change->data.msg.message); + change->data.msg.message = NULL; + break;Hm, this will have some overhead, but I guess the messages won't be
super frequent, and usually not very large.
Yeah but since we don't really know the size of the future messages it's
hard to have some preallocated buffer for this so I dunno how else to do it.
+/* + * Queue message into a transaction so it can be processed upon commit. + */ +void +ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn, + const char *prefix, Size msg_sz, const char *msg) +{ + ReorderBufferChange *change; + + Assert(xid != InvalidTransactionId); + + change = ReorderBufferGetChange(rb); + change->action = REORDER_BUFFER_CHANGE_MESSAGE; + change->data.msg.prefix = pstrdup(prefix); + change->data.msg.message_size = msg_sz; + change->data.msg.message = palloc(msg_sz); + memcpy(change->data.msg.message, msg, msg_sz); + + ReorderBufferQueueChange(rb, xid, lsn, change); +}I'm not sure right now if there's any guarantee that the current memory
context is meaningful here? IIRC other long-lived allocations explicitly
use a context?
I didn't find any explicit guarantee so I added one.
+ case REORDER_BUFFER_CHANGE_MESSAGE: + { + char *data; + size_t prefix_size = strlen(change->data.msg.prefix) + 1; + + sz += prefix_size + change->data.msg.message_size; + ReorderBufferSerializeReserve(rb, sz); + + data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange); + memcpy(data, change->data.msg.prefix, + prefix_size); + memcpy(data + prefix_size, change->data.msg.message, + change->data.msg.message_size); + break; + }Can you please include the sizes of the blocks explicitly, rather than
relying on 0 termination?
Okay, I see I did that in WAL, no idea why I didn't do the same here.
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL) PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL) PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL) +PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)Did you consider doing this via the standby rmgr instead?
Yes in one of the first versions I did that but Simon didn't like that
in his review as this has nothing to do with standby.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-22.patchtext/x-diff; name=logical-messages-2016-03-22.patchDownload
From 77fce7aaf550186c853dd6c2eaba31458af13be5 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++++--
contrib/test_decoding/expected/messages.out | 56 +++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 +++++
contrib/test_decoding/test_decoding.c | 17 +++++
doc/src/sgml/func.sgml | 45 ++++++++++++
doc/src/sgml/logicaldecoding.sgml | 34 +++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 +++++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 65 +++++++++++++++++
src/backend/replication/logical/logical.c | 37 ++++++++++
src/backend/replication/logical/logicalfuncs.c | 27 +++++++
src/backend/replication/logical/message.c | 85 ++++++++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 93 +++++++++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 +++++
src/bin/pg_xlogdump/.gitignore | 20 +-----
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 ++
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 +++++++++++
src/include/replication/output_plugin.h | 12 ++++
src/include/replication/reorderbuffer.h | 20 ++++++
src/include/replication/snapbuild.h | 2 +
27 files changed, 641 insertions(+), 31 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..1997774 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,9 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ const char *prefix, Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +86,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +476,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ txn != NULL, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index ae93e69..7ac71d1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18238,6 +18238,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..7c9d03a 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,39 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>ctx</parameter> and <parameter>txn</parameter> parameters
+ have the same contents as for the <function>begin_cb</function> with the
+ difference that <parameter>txn</parameter> is null for non-transactional
+ messages. The <parameter>lsn</parameter> has WAL position of the message.
+ The <parameter>prefix</parameter> is arbitrary textual prefix which can
+ be used for identifying interesting messages for the current plugin.
+ And finally the <parameter>message</parameter> parameter holds the
+ actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..30ccf07 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,65 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(r), buf->origptr);
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ /*
+ * Transactional messages goe to RecordBuffer, while non-transactional
+ * ones get sent to output plugin immediately.
+ */
+ if (message->transactional)
+ {
+ if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr))
+ return;
+
+ ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r),
+ buf->endptr,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+ }
+ else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT &&
+ !SnapBuildXactNeedsSkip(builder, buf->origptr))
+ {
+ volatile Snapshot snapshot_now;
+ ReorderBuffer *rb = ctx->reorder;
+
+ /* setup snapshot to allow catalog access */
+ snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r));
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ PG_TRY();
+ {
+ rb->message(rb, NULL, buf->origptr, message->message,
+ message->message_size,
+ message->message + message->prefix_size);
+
+ TeardownHistoricSnapshot(false);
+ }
+ PG_CATCH();
+ {
+ TeardownHistoricSnapshot(true);
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+ }
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..7863d52 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,39 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, const char *prefix, Size sz,
+ const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, prefix, sz, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..552dac3 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..85c597b
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,85 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The plugin authors must take extra care to use unique prefix,
+ * good options seems to be for example to use the name of the extension.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..7894971 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,33 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg)
+{
+ MemoryContext oldcontext;
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ oldcontext = MemoryContextSwitchTo(rb->context);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = msg_sz;
+ change->data.msg.message = palloc(msg_sz);
+ memcpy(change->data.msg.message, msg, msg_sz);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1528,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2201,33 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ Size prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size +
+ sizeof(Size) + sizeof(Size);
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+
+ /* write the prefix including the size */
+ memcpy(data, &prefix_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ data += prefix_size;
+
+ /* write the message including the size */
+ memcpy(data, &change->data.msg.message_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.message,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2484,30 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size prefix_size;
+
+ /* read prefix */
+ memcpy(&prefix_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ Assert(change->data.msg.prefix[prefix_size-1] == '\0');
+ data += prefix_size;
+
+ /* read the messsage */
+ memcpy(&change->data.msg.message_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ change->data.msg.message_size);
+ memcpy(change->data.msg.message, data,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/.gitignore b/src/bin/pg_xlogdump/.gitignore
index eebaf30..c4783f1 100644
--- a/src/bin/pg_xlogdump/.gitignore
+++ b/src/bin/pg_xlogdump/.gitignore
@@ -1,22 +1,4 @@
/pg_xlogdump
# Source files copied from src/backend/access/rmgrdesc/
-/brindesc.c
-/clogdesc.c
-/committsdesc.c
-/dbasedesc.c
-/gindesc.c
-/gistdesc.c
-/hashdesc.c
-/heapdesc.c
-/mxactdesc.c
-/nbtdesc.c
-/relmapdesc.c
-/replorigindesc.c
-/seqdesc.c
-/smgrdesc.c
-/spgdesc.c
-/standbydesc.c
-/tblspcdesc.c
-/xactdesc.c
-/xlogdesc.c
+/*desc.c
/xlogreader.c
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..3713739 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5114,6 +5114,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..b5c1ebf
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ size_t prefix_size; /* length of prefix */
+ size_t message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefix of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..fb5b758 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,17 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +107,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..37c6c4b 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ Size message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,14 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +317,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +368,8 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ const char *prefix, Size msg_sz, const char *msg);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
On 2016-03-22 14:03:06 +0100, Petr Jelinek wrote:
On 22/03/16 12:47, Andres Freund wrote:
On 2016-03-21 18:10:55 +0100, Petr Jelinek wrote:
+ + <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + const char *prefix, + Size message_size, + const char *message +);I see you removed the transactional parameter. I'm doubtful that that's
a good idea: It seems like it'd be rather helpful to pass the
transaction for a nontransaction message that's emitted while an xid was
assigned?Hmm but won't that give the output plugin even transactions that were later
aborted? That seems quite different behavior from how the txn parameter
works everywhere else.
Seems harmless to me if called out.
+/* + * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer(). + */ +static void +DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf) +{ + SnapBuild *builder = ctx->snapshot_builder; + XLogReaderState *r = buf->record; + uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK; + xl_logical_message *message; + + if (info != XLOG_LOGICAL_MESSAGE) + elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info); + + message = (xl_logical_message *) XLogRecGetData(r); + + if (message->transactional) + { + if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr)) + return; + + ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r), + buf->endptr, + message->message, /* first part of message is prefix */ + message->message_size, + message->message + message->prefix_size); + } + else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT && + !SnapBuildXactNeedsSkip(builder, buf->origptr)) + { + volatile Snapshot snapshot_now; + ReorderBuffer *rb = ctx->reorder; + + /* setup snapshot to allow catalog access */ + snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r)); + SetupHistoricSnapshot(snapshot_now, NULL); + rb->message(rb, NULL, buf->origptr, message->message, + message->message_size, + message->message + message->prefix_size); + TeardownHistoricSnapshot(false); + } +}A number of things:
1) The SnapBuildProcessChange needs to be toplevel, not just for
transactional messages - we can't yet necessarily build a snapshot.Nope, the snapshot state is checked in the else if.
2) I'm inclined to move even the non-transactional stuff to reorderbuffer.
Well, it's not doing anything with reorderbuffer but sure it can be done
(didn't do it in the attached though).
I think there'll be some interactions if we ever do some parts in
parallel and such. I'd rather keep decode.c to only do the lowest level
stuff.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 22/03/16 14:11, Andres Freund wrote:
On 2016-03-22 14:03:06 +0100, Petr Jelinek wrote:
On 22/03/16 12:47, Andres Freund wrote:
On 2016-03-21 18:10:55 +0100, Petr Jelinek wrote:
+ + <sect3 id="logicaldecoding-output-plugin-message"> + <title>Generic Message Callback</title> + + <para> + The optional <function>message_cb</function> callback is called whenever + a logical decoding message has been decoded. +<programlisting> +typedef void (*LogicalDecodeMessageCB) ( + struct LogicalDecodingContext *, + ReorderBufferTXN *txn, + XLogRecPtr message_lsn, + const char *prefix, + Size message_size, + const char *message +);I see you removed the transactional parameter. I'm doubtful that that's
a good idea: It seems like it'd be rather helpful to pass the
transaction for a nontransaction message that's emitted while an xid was
assigned?Hmm but won't that give the output plugin even transactions that were later
aborted? That seems quite different behavior from how the txn parameter
works everywhere else.Seems harmless to me if called out.
All right, after some consideration I agree.
+/* + * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer(). + */ +static void +DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf) +{ + SnapBuild *builder = ctx->snapshot_builder; + XLogReaderState *r = buf->record; + uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK; + xl_logical_message *message; + + if (info != XLOG_LOGICAL_MESSAGE) + elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info); + + message = (xl_logical_message *) XLogRecGetData(r); + + if (message->transactional) + { + if (!SnapBuildProcessChange(builder, XLogRecGetXid(r), buf->origptr)) + return; + + ReorderBufferQueueMessage(ctx->reorder, XLogRecGetXid(r), + buf->endptr, + message->message, /* first part of message is prefix */ + message->message_size, + message->message + message->prefix_size); + } + else if (SnapBuildCurrentState(builder) == SNAPBUILD_CONSISTENT && + !SnapBuildXactNeedsSkip(builder, buf->origptr)) + { + volatile Snapshot snapshot_now; + ReorderBuffer *rb = ctx->reorder; + + /* setup snapshot to allow catalog access */ + snapshot_now = SnapBuildGetOrBuildSnapshot(builder, XLogRecGetXid(r)); + SetupHistoricSnapshot(snapshot_now, NULL); + rb->message(rb, NULL, buf->origptr, message->message, + message->message_size, + message->message + message->prefix_size); + TeardownHistoricSnapshot(false); + } +}A number of things:
1) The SnapBuildProcessChange needs to be toplevel, not just for
transactional messages - we can't yet necessarily build a snapshot.Nope, the snapshot state is checked in the else if.
2) I'm inclined to move even the non-transactional stuff to reorderbuffer.
Well, it's not doing anything with reorderbuffer but sure it can be done
(didn't do it in the attached though).I think there'll be some interactions if we ever do some parts in
parallel and such. I'd rather keep decode.c to only do the lowest level
stuff.
Did it that way but I do like the resulting code less.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-23.patchtext/x-diff; name=logical-messages-2016-03-23.patchDownload
From 6076cb5d85cab4cfe4cb99499ed49e5a0c384033 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++--
contrib/test_decoding/expected/messages.out | 56 +++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 17 ++++
contrib/test_decoding/test_decoding.c | 18 ++++
doc/src/sgml/func.sgml | 45 +++++++++
doc/src/sgml/logicaldecoding.sgml | 38 ++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 46 +++++++++
src/backend/replication/logical/logical.c | 38 ++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++
src/backend/replication/logical/message.c | 85 +++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 121 ++++++++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++
src/bin/pg_xlogdump/.gitignore | 20 +---
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 +
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++
src/include/replication/output_plugin.h | 13 +++
src/include/replication/reorderbuffer.h | 22 +++++
src/include/replication/snapbuild.h | 2 +
27 files changed, 659 insertions(+), 31 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..7fa8256
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,56 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 1 prefix: test, sz: 4 content:msg3
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+(5 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..8744eb6
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,17 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..3336e1e 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,10 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn, bool transactional,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ transactional, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index ae93e69..7ac71d1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18238,6 +18238,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..dd67c8f 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,43 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>txn</parameter> parameter contains meta information about
+ the transaction, like the time stamp at which it has been committed and
+ its XID. Note however that it can be NULL when the message is
+ non-transactional and the XID was not assigned yet in the transaction
+ which logged the message. The <parameter>lsn</parameter> has WAL
+ position of the message. The <parameter>transactional</parameter> says
+ if he message was sent as transactional or not.
+ The <parameter>prefix</parameter> is arbitrary null-terminated prefix
+ which can be used for identifying interesting messages for the current
+ plugin. And finally the <parameter>message</parameter> parameter holds
+ the actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..01c96f7 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,46 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ TransactionId xid = XLogRecGetXid(r);
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ Snapshot snapshot;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(r), buf->origptr);
+
+ /* No point in doing anything yet. */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, xid, buf->origptr))
+ return;
+ else if (!message->transactional &&
+ (SnapBuildCurrentState(builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ return;
+
+ snapshot = SnapBuildGetOrBuildSnapshot(builder, xid);
+ ReorderBufferQueueMessage(ctx->reorder, xid, snapshot, buf->endptr,
+ message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..c06b2fa 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional, prefix,
+ message_size, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..552dac3 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..85c597b
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,85 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The plugin authors must take extra care to use unique prefix,
+ * good options seems to be for example to use the name of the extension.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..1420ddd 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,61 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
+ Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message)
+{
+ if (transactional)
+ {
+ MemoryContext oldcontext;
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ oldcontext = MemoryContextSwitchTo(rb->context);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = message_size;
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, message, message_size);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+
+ MemoryContextSwitchTo(oldcontext);
+ }
+ else
+ {
+ ReorderBufferTXN *txn = NULL;
+ volatile Snapshot snapshot_now = snapshot;
+
+ if (xid != InvalidTransactionId)
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ /* setup snapshot to allow catalog access */
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ PG_TRY();
+ {
+ rb->message(rb, txn, lsn, false, prefix, message_size, message);
+
+ TeardownHistoricSnapshot(false);
+ }
+ PG_CATCH();
+ {
+ TeardownHistoricSnapshot(true);
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1556,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn, true,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2229,33 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ Size prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size +
+ sizeof(Size) + sizeof(Size);
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+
+ /* write the prefix including the size */
+ memcpy(data, &prefix_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ data += prefix_size;
+
+ /* write the message including the size */
+ memcpy(data, &change->data.msg.message_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.message,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2512,30 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size prefix_size;
+
+ /* read prefix */
+ memcpy(&prefix_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ Assert(change->data.msg.prefix[prefix_size-1] == '\0');
+ data += prefix_size;
+
+ /* read the messsage */
+ memcpy(&change->data.msg.message_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ change->data.msg.message_size);
+ memcpy(change->data.msg.message, data,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/.gitignore b/src/bin/pg_xlogdump/.gitignore
index eebaf30..c4783f1 100644
--- a/src/bin/pg_xlogdump/.gitignore
+++ b/src/bin/pg_xlogdump/.gitignore
@@ -1,22 +1,4 @@
/pg_xlogdump
# Source files copied from src/backend/access/rmgrdesc/
-/brindesc.c
-/clogdesc.c
-/committsdesc.c
-/dbasedesc.c
-/gindesc.c
-/gistdesc.c
-/hashdesc.c
-/heapdesc.c
-/mxactdesc.c
-/nbtdesc.c
-/relmapdesc.c
-/replorigindesc.c
-/seqdesc.c
-/smgrdesc.c
-/spgdesc.c
-/standbydesc.c
-/tblspcdesc.c
-/xactdesc.c
-/xlogdesc.c
+/*desc.c
/xlogreader.c
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..3713739 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5114,6 +5114,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..8b968d5
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ Size prefix_size; /* length of prefix */
+ Size message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefix of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..4c54953 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ Size message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +318,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +369,9 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
Petr Jelinek wrote:
+++ b/contrib/test_decoding/sql/messages.sql @@ -0,0 +1,17 @@ +-- predictability +SET synchronous_commit = on; + +SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding'); + +SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1'); +SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2'); + +BEGIN; +SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3'); +SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4'); +SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5'); +COMMIT; + +SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1'); + +SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
No tests for a rolled back transaction?
--
�lvaro Herrera http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 23/03/16 14:17, Alvaro Herrera wrote:
Petr Jelinek wrote:
+++ b/contrib/test_decoding/sql/messages.sql @@ -0,0 +1,17 @@ +-- predictability +SET synchronous_commit = on; + +SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding'); + +SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1'); +SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2'); + +BEGIN; +SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3'); +SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4'); +SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5'); +COMMIT; + +SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1'); + +SELECT 'init' FROM pg_drop_replication_slot('regression_slot');No tests for a rolled back transaction?
Good point, probably worth testing especially since we have
non-transactional messages.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-03-23-2.patchtext/plain; charset=UTF-8; name=logical-messages-2016-03-23-2.patchDownload
From 81fc28cedc19fe0f91f882d42989c14113a40f88 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++--
contrib/test_decoding/expected/messages.out | 71 ++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 22 +++++
contrib/test_decoding/test_decoding.c | 18 ++++
doc/src/sgml/func.sgml | 45 +++++++++
doc/src/sgml/logicaldecoding.sgml | 38 ++++++++
src/backend/access/rmgrdesc/Makefile | 4 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 46 +++++++++
src/backend/replication/logical/logical.c | 38 ++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++
src/backend/replication/logical/message.c | 85 +++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 121 ++++++++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++
src/bin/pg_xlogdump/.gitignore | 20 +---
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 +
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++
src/include/replication/output_plugin.h | 13 +++
src/include/replication/reorderbuffer.h | 22 +++++
src/include/replication/snapbuild.h | 2 +
27 files changed, 679 insertions(+), 31 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..2edbf2f
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,71 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+ROLLBACK;
+BEGIN;
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+SELECT 'msg6' FROM pg_logical_emit_message(false, 'test', 'msg6');
+ ?column?
+----------
+ msg6
+(1 row)
+
+SELECT 'msg7' FROM pg_logical_emit_message(true, 'test', 'msg7');
+ ?column?
+----------
+ msg7
+(1 row)
+
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 0 prefix: test, sz: 4 content:msg6
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+ message: transactional: 1 prefix: test, sz: 4 content:msg7
+(6 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..37bd100
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,22 @@
+-- predictability
+SET synchronous_commit = on;
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ROLLBACK;
+
+BEGIN;
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+SELECT 'msg6' FROM pg_logical_emit_message(false, 'test', 'msg6');
+SELECT 'msg7' FROM pg_logical_emit_message(true, 'test', 'msg7');
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..3336e1e 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,10 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn, bool transactional,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ transactional, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index ae93e69..7ac71d1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18238,6 +18238,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..dd67c8f 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,43 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>txn</parameter> parameter contains meta information about
+ the transaction, like the time stamp at which it has been committed and
+ its XID. Note however that it can be NULL when the message is
+ non-transactional and the XID was not assigned yet in the transaction
+ which logged the message. The <parameter>lsn</parameter> has WAL
+ position of the message. The <parameter>transactional</parameter> says
+ if he message was sent as transactional or not.
+ The <parameter>prefix</parameter> is arbitrary null-terminated prefix
+ which can be used for identifying interesting messages for the current
+ plugin. And finally the <parameter>message</parameter> parameter holds
+ the actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c72a1f2..723b4d8 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o gindesc.o gistdesc.o \
- hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o relmapdesc.o \
- replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
+ hashdesc.o heapdesc.o logicalmsgdesc.o mxactdesc.o nbtdesc.o \
+ relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7c4d773..1a42121 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -23,6 +23,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 13af485..01c96f7 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -457,6 +463,46 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ TransactionId xid = XLogRecGetXid(r);
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ Snapshot snapshot;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(r), buf->origptr);
+
+ /* No point in doing anything yet. */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, xid, buf->origptr))
+ return;
+ else if (!message->transactional &&
+ (SnapBuildCurrentState(builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ return;
+
+ snapshot = SnapBuildGetOrBuildSnapshot(builder, xid);
+ ReorderBufferQueueMessage(ctx->reorder, xid, snapshot, buf->endptr,
+ message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..c06b2fa 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional, prefix,
+ message_size, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index f789fc1..552dac3 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -363,3 +366,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..85c597b
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,85 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The plugin authors must take extra care to use unique prefix,
+ * good options seems to be for example to use the name of the extension.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index f2b8f4b..1420ddd 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,61 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
+ Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message)
+{
+ if (transactional)
+ {
+ MemoryContext oldcontext;
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ oldcontext = MemoryContextSwitchTo(rb->context);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = message_size;
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, message, message_size);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+
+ MemoryContextSwitchTo(oldcontext);
+ }
+ else
+ {
+ ReorderBufferTXN *txn = NULL;
+ volatile Snapshot snapshot_now = snapshot;
+
+ if (xid != InvalidTransactionId)
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ /* setup snapshot to allow catalog access */
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ PG_TRY();
+ {
+ rb->message(rb, txn, lsn, false, prefix, message_size, message);
+
+ TeardownHistoricSnapshot(false);
+ }
+ PG_CATCH();
+ {
+ TeardownHistoricSnapshot(true);
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1556,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn, true,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2229,33 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ Size prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size +
+ sizeof(Size) + sizeof(Size);
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+
+ /* write the prefix including the size */
+ memcpy(data, &prefix_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ data += prefix_size;
+
+ /* write the message including the size */
+ memcpy(data, &change->data.msg.message_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.message,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2512,30 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size prefix_size;
+
+ /* read prefix */
+ memcpy(&prefix_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ Assert(change->data.msg.prefix[prefix_size-1] == '\0');
+ data += prefix_size;
+
+ /* read the messsage */
+ memcpy(&change->data.msg.message_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ change->data.msg.message_size);
+ memcpy(change->data.msg.message, data,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/.gitignore b/src/bin/pg_xlogdump/.gitignore
index eebaf30..c4783f1 100644
--- a/src/bin/pg_xlogdump/.gitignore
+++ b/src/bin/pg_xlogdump/.gitignore
@@ -1,22 +1,4 @@
/pg_xlogdump
# Source files copied from src/backend/access/rmgrdesc/
-/brindesc.c
-/clogdesc.c
-/committsdesc.c
-/dbasedesc.c
-/gindesc.c
-/gistdesc.c
-/hashdesc.c
-/heapdesc.c
-/mxactdesc.c
-/nbtdesc.c
-/relmapdesc.c
-/replorigindesc.c
-/seqdesc.c
-/smgrdesc.c
-/spgdesc.c
-/standbydesc.c
-/tblspcdesc.c
-/xactdesc.c
-/xlogdesc.c
+/*desc.c
/xlogreader.c
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index f9cd395..6ba7f22 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -25,6 +25,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index fab912d..35c242d 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -45,3 +45,4 @@ PG_RMGR(RM_SPGIST_ID, "SPGist", spg_redo, spg_desc, spg_identify, spg_xlog_start
PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..3713739 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5114,6 +5114,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..8b968d5
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ Size prefix_size; /* length of prefix */
+ Size message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefix of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..4c54953 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ Size message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +318,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +369,9 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
Hi,
I rebased this patch on top of current master as the generic wal commit
added some conflicting changes. Also fixed couple of typos in comments
and added non ascii message to test.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-2016-04-04.patchtext/x-diff; name=logical-messages-2016-04-04.patchDownload
From b74e6dc5956446d514130c575263f4ba6ad71db3 Mon Sep 17 00:00:00 2001
From: Petr Jelinek <pjmodos@pjmodos.net>
Date: Wed, 24 Feb 2016 17:02:36 +0100
Subject: [PATCH] Logical Decoding Messages
---
contrib/test_decoding/Makefile | 2 +-
contrib/test_decoding/expected/ddl.out | 21 ++--
contrib/test_decoding/expected/messages.out | 79 ++++++++++++++++
contrib/test_decoding/sql/ddl.sql | 3 +-
contrib/test_decoding/sql/messages.sql | 25 +++++
contrib/test_decoding/test_decoding.c | 18 ++++
doc/src/sgml/func.sgml | 45 +++++++++
doc/src/sgml/logicaldecoding.sgml | 38 ++++++++
src/backend/access/rmgrdesc/Makefile | 6 +-
src/backend/access/rmgrdesc/logicalmsgdesc.c | 41 ++++++++
src/backend/access/transam/rmgr.c | 1 +
src/backend/replication/logical/Makefile | 2 +-
src/backend/replication/logical/decode.c | 46 +++++++++
src/backend/replication/logical/logical.c | 38 ++++++++
src/backend/replication/logical/logicalfuncs.c | 27 ++++++
src/backend/replication/logical/message.c | 87 +++++++++++++++++
src/backend/replication/logical/reorderbuffer.c | 121 ++++++++++++++++++++++++
src/backend/replication/logical/snapbuild.c | 19 ++++
src/bin/pg_xlogdump/.gitignore | 21 +---
src/bin/pg_xlogdump/rmgrdesc.c | 1 +
src/include/access/rmgrlist.h | 1 +
src/include/catalog/pg_proc.h | 4 +
src/include/replication/logicalfuncs.h | 2 +
src/include/replication/message.h | 41 ++++++++
src/include/replication/output_plugin.h | 13 +++
src/include/replication/reorderbuffer.h | 22 +++++
src/include/replication/snapbuild.h | 2 +
27 files changed, 693 insertions(+), 33 deletions(-)
create mode 100644 contrib/test_decoding/expected/messages.out
create mode 100644 contrib/test_decoding/sql/messages.sql
create mode 100644 src/backend/access/rmgrdesc/logicalmsgdesc.c
create mode 100644 src/backend/replication/logical/message.c
create mode 100644 src/include/replication/message.h
diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index 06c9546..309cb0b 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -38,7 +38,7 @@ submake-test_decoding:
$(MAKE) -C $(top_builddir)/contrib/test_decoding
REGRESSCHECKS=ddl xact rewrite toast permissions decoding_in_xact \
- decoding_into_rel binary prepared replorigin time
+ decoding_into_rel binary prepared replorigin time messages
regresscheck: | submake-regress submake-test_decoding temp-install
$(MKDIR_P) regression_output
diff --git a/contrib/test_decoding/expected/ddl.out b/contrib/test_decoding/expected/ddl.out
index 77719e8..32cd24d 100644
--- a/contrib/test_decoding/expected/ddl.out
+++ b/contrib/test_decoding/expected/ddl.out
@@ -220,11 +220,17 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'inc
(7 rows)
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
+ ?column?
+----------------
+ tx logical msg
+(1 row)
+
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
@@ -233,12 +239,13 @@ SELECT count(*), min(data), max(data)
FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
GROUP BY substring(data, 1, 24)
ORDER BY 1,2;
- count | min | max
--------+-------------------------------------------------+------------------------------------------------------------------------
- 1 | BEGIN | BEGIN
- 1 | COMMIT | COMMIT
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(3 rows)
+ count | min | max
+-------+-----------------------------------------------------------------------+------------------------------------------------------------------------
+ 1 | BEGIN | BEGIN
+ 1 | COMMIT | COMMIT
+ 1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
+ 20467 | table public.tr_etoomuch: DELETE: id[integer]:1 | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
+(4 rows)
-- check updates of primary keys work correctly
BEGIN;
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
new file mode 100644
index 0000000..e26f793
--- /dev/null
+++ b/contrib/test_decoding/expected/messages.out
@@ -0,0 +1,79 @@
+-- predictability
+SET synchronous_commit = on;
+SET client_encoding = 'utf8';
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column?
+----------
+ init
+(1 row)
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+ ?column?
+----------
+ msg1
+(1 row)
+
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+ ?column?
+----------
+ msg2
+(1 row)
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+ ?column?
+----------
+ msg3
+(1 row)
+
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ ?column?
+----------
+ msg4
+(1 row)
+
+ROLLBACK;
+BEGIN;
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+ ?column?
+----------
+ msg5
+(1 row)
+
+SELECT 'msg6' FROM pg_logical_emit_message(false, 'test', 'msg6');
+ ?column?
+----------
+ msg6
+(1 row)
+
+SELECT 'msg7' FROM pg_logical_emit_message(true, 'test', 'msg7');
+ ?column?
+----------
+ msg7
+(1 row)
+
+COMMIT;
+SELECT 'žluÅ¥ouÄký kůÅ' FROM pg_logical_emit_message(true, 'test', 'žluÅ¥ouÄký kůÅ');
+ ?column?
+---------------
+ žluÅ¥ouÄký kůÅ
+(1 row)
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+----------------------------------------------------------------------
+ message: transactional: 1 prefix: test, sz: 4 content:msg1
+ message: transactional: 0 prefix: test, sz: 4 content:msg2
+ message: transactional: 0 prefix: test, sz: 4 content:msg4
+ message: transactional: 0 prefix: test, sz: 4 content:msg6
+ message: transactional: 1 prefix: test, sz: 4 content:msg5
+ message: transactional: 1 prefix: test, sz: 4 content:msg7
+ message: transactional: 1 prefix: test, sz: 19 content:žluÅ¥ouÄký kůÅ
+(7 rows)
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ init
+(1 row)
+
diff --git a/contrib/test_decoding/sql/ddl.sql b/contrib/test_decoding/sql/ddl.sql
index ad928ad..b1f7bf6 100644
--- a/contrib/test_decoding/sql/ddl.sql
+++ b/contrib/test_decoding/sql/ddl.sql
@@ -108,11 +108,12 @@ DELETE FROM tr_pkey;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
/*
- * check that disk spooling works
+ * check that disk spooling works (also for logical messages)
*/
BEGIN;
CREATE TABLE tr_etoomuch (id serial primary key, data int);
INSERT INTO tr_etoomuch(data) SELECT g.i FROM generate_series(1, 10234) g(i);
+SELECT 'tx logical msg' FROM pg_logical_emit_message(true, 'test', 'tx logical msg');
DELETE FROM tr_etoomuch WHERE id < 5000;
UPDATE tr_etoomuch SET data = - data WHERE id > 5000;
COMMIT;
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
new file mode 100644
index 0000000..d72191b
--- /dev/null
+++ b/contrib/test_decoding/sql/messages.sql
@@ -0,0 +1,25 @@
+-- predictability
+SET synchronous_commit = on;
+SET client_encoding = 'utf8';
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+SELECT 'msg1' FROM pg_logical_emit_message(true, 'test', 'msg1');
+SELECT 'msg2' FROM pg_logical_emit_message(false, 'test', 'msg2');
+
+BEGIN;
+SELECT 'msg3' FROM pg_logical_emit_message(true, 'test', 'msg3');
+SELECT 'msg4' FROM pg_logical_emit_message(false, 'test', 'msg4');
+ROLLBACK;
+
+BEGIN;
+SELECT 'msg5' FROM pg_logical_emit_message(true, 'test', 'msg5');
+SELECT 'msg6' FROM pg_logical_emit_message(false, 'test', 'msg6');
+SELECT 'msg7' FROM pg_logical_emit_message(true, 'test', 'msg7');
+COMMIT;
+
+SELECT 'žluÅ¥ouÄký kůÅ' FROM pg_logical_emit_message(true, 'test', 'žluÅ¥ouÄký kůÅ');
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 4cf808f..3336e1e 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -21,6 +21,7 @@
#include "replication/output_plugin.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "utils/builtins.h"
@@ -63,6 +64,10 @@ static void pg_decode_change(LogicalDecodingContext *ctx,
ReorderBufferChange *change);
static bool pg_decode_filter(LogicalDecodingContext *ctx,
RepOriginId origin_id);
+static void pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr message_lsn,
+ bool transactional, const char *prefix,
+ Size sz, const char *message);
void
_PG_init(void)
@@ -82,6 +87,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
cb->commit_cb = pg_decode_commit_txn;
cb->filter_by_origin_cb = pg_decode_filter;
cb->shutdown_cb = pg_decode_shutdown;
+ cb->message_cb = pg_decode_message;
}
@@ -471,3 +477,15 @@ pg_decode_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginWrite(ctx, true);
}
+
+static void
+pg_decode_message(LogicalDecodingContext *ctx,
+ ReorderBufferTXN *txn, XLogRecPtr lsn, bool transactional,
+ const char *prefix, Size sz, const char *message)
+{
+ OutputPluginPrepareWrite(ctx, true);
+ appendStringInfo(ctx->out, "message: transactional: %d prefix: %s, sz: %zu content:",
+ transactional, prefix, sz);
+ appendBinaryStringInfo(ctx->out, message, sz);
+ OutputPluginWrite(ctx, true);
+}
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 1bc9fbc..14b686f 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -18237,6 +18237,51 @@ postgres=# SELECT * FROM pg_xlogfile_name_offset(pg_stop_backup());
</entry>
</row>
+ <row>
+ <entry id="pg-logical-emit-message-text">
+ <indexterm>
+ <primary>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>text</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit text logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the text of the message.
+ </entry>
+ </row>
+
+ <row>
+ <entry id="pg-logical-emit-message-bytea">
+ <indexterm>
+ <primary>>pg_logical_emit_message</primary>
+ </indexterm>
+ <literal><function>>pg_logical_emit_message(<parameter>transactional</parameter> <type>bool</type>, <parameter>prefix</parameter> <type>text</type>, <parameter>content</parameter> <type>bytea</type>)</function></literal>
+ </entry>
+ <entry>
+ void
+ </entry>
+ <entry>
+ Emit binary logical decoding message. This can be used to pass generic
+ messages to logical decoding plugins through WAL. The parameter
+ <parameter>transactional</parameter> specifies if the message should
+ be part of current transaction or if it should be written immediately
+ and decoded as soon as the logical decoding reads the record. The
+ <parameter>prefix</parameter> is textual prefix used by the logical
+ decoding plugins to easily recognize interesting messages for them.
+ The <parameter>content</parameter> is the binary content of the
+ message.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 45fdfeb..8306d9f 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -363,6 +363,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
@@ -602,6 +603,43 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (
more efficient.
</para>
</sect3>
+
+ <sect3 id="logicaldecoding-output-plugin-message">
+ <title>Generic Message Callback</title>
+
+ <para>
+ The optional <function>message_cb</function> callback is called whenever
+ a logical decoding message has been decoded.
+<programlisting>
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message
+);
+</programlisting>
+ The <parameter>txn</parameter> parameter contains meta information about
+ the transaction, like the time stamp at which it has been committed and
+ its XID. Note however that it can be NULL when the message is
+ non-transactional and the XID was not assigned yet in the transaction
+ which logged the message. The <parameter>lsn</parameter> has WAL
+ position of the message. The <parameter>transactional</parameter> says
+ if the message was sent as transactional or not.
+ The <parameter>prefix</parameter> is arbitrary null-terminated prefix
+ which can be used for identifying interesting messages for the current
+ plugin. And finally the <parameter>message</parameter> parameter holds
+ the actual message of <parameter>message_size</parameter> size.
+ </para>
+ <para>
+ Extra care should be taken to ensure that the prefix the output plugin
+ considers interesting is unique. Using name of the extension or the
+ output plugin itself is often a good choice.
+ </para>
+ </sect3>
+
</sect2>
<sect2 id="logicaldecoding-output-plugin-output">
diff --git a/src/backend/access/rmgrdesc/Makefile b/src/backend/access/rmgrdesc/Makefile
index c0e38fd..5514db1 100644
--- a/src/backend/access/rmgrdesc/Makefile
+++ b/src/backend/access/rmgrdesc/Makefile
@@ -9,8 +9,8 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = brindesc.o clogdesc.o committsdesc.o dbasedesc.o genericdesc.o \
- gindesc.o gistdesc.o hashdesc.o heapdesc.o mxactdesc.o nbtdesc.o \
- relmapdesc.o replorigindesc.o seqdesc.o smgrdesc.o spgdesc.o \
- standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
+ gindesc.o gistdesc.o hashdesc.o heapdesc.o logicalmsgdesc.o \
+ mxactdesc.o nbtdesc.o relmapdesc.o replorigindesc.o seqdesc.o \
+ smgrdesc.o spgdesc.o standbydesc.o tblspcdesc.o xactdesc.o xlogdesc.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/rmgrdesc/logicalmsgdesc.c b/src/backend/access/rmgrdesc/logicalmsgdesc.c
new file mode 100644
index 0000000..b194e14
--- /dev/null
+++ b/src/backend/access/rmgrdesc/logicalmsgdesc.c
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ *
+ * logicalmsgdesc.c
+ * rmgr descriptor routines for replication/logical/message.c
+ *
+ * Portions Copyright (c) 2015-2016, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/rmgrdesc/logicalmsgdesc.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "replication/message.h"
+
+void
+logicalmsg_desc(StringInfo buf, XLogReaderState *record)
+{
+ char *rec = XLogRecGetData(record);
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info == XLOG_LOGICAL_MESSAGE)
+ {
+ xl_logical_message *xlrec = (xl_logical_message *) rec;
+
+ appendStringInfo(buf, "%s message size %zu bytes",
+ xlrec->transactional ? "transactional" : "nontransactional",
+ xlrec->message_size);
+ }
+}
+
+const char *
+logicalmsg_identify(uint8 info)
+{
+ if ((info & ~XLR_INFO_MASK) == XLOG_LOGICAL_MESSAGE)
+ return "MESSAGE";
+
+ return NULL;
+}
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c
index 7b38c16..31c5fd1 100644
--- a/src/backend/access/transam/rmgr.c
+++ b/src/backend/access/transam/rmgr.c
@@ -24,6 +24,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "storage/standby.h"
#include "utils/relmapper.h"
diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 8adea13..1d7ca06 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
-OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o origin.o \
+OBJS = decode.o logical.o logicalfuncs.o message.o origin.o reorderbuffer.o \
snapbuild.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 7781ebc..3e80c4a 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -39,6 +39,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
+#include "replication/message.h"
#include "replication/reorderbuffer.h"
#include "replication/origin.h"
#include "replication/snapbuild.h"
@@ -58,6 +59,7 @@ static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
/* individual record(group)'s handlers */
static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -123,6 +125,10 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
DecodeHeapOp(ctx, &buf);
break;
+ case RM_LOGICALMSG_ID:
+ DecodeLogicalMsgOp(ctx, &buf);
+ break;
+
/*
* Rmgrs irrelevant for logical decoding; they describe stuff not
* represented in logical decoding. Add new rmgrs in rmgrlist.h's
@@ -458,6 +464,46 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+/*
+ * Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogReaderState *r = buf->record;
+ TransactionId xid = XLogRecGetXid(r);
+ uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ Snapshot snapshot;
+ xl_logical_message *message;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(ERROR, "unexpected RM_LOGICALMSG_ID record type: %u", info);
+
+ ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(r), buf->origptr);
+
+ /* No point in doing anything yet. */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ message = (xl_logical_message *) XLogRecGetData(r);
+
+ if (message->transactional &&
+ !SnapBuildProcessChange(builder, xid, buf->origptr))
+ return;
+ else if (!message->transactional &&
+ (SnapBuildCurrentState(builder) != SNAPBUILD_CONSISTENT ||
+ SnapBuildXactNeedsSkip(builder, buf->origptr)))
+ return;
+
+ snapshot = SnapBuildGetOrBuildSnapshot(builder, xid);
+ ReorderBufferQueueMessage(ctx->reorder, xid, snapshot, buf->endptr,
+ message->transactional,
+ message->message, /* first part of message is prefix */
+ message->message_size,
+ message->message + message->prefix_size);
+}
+
static inline bool
FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 2e6d3f9..c06b2fa 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -62,6 +62,9 @@ static void commit_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
static void change_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
+static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message);
static void LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin);
@@ -178,6 +181,7 @@ StartupDecodingContext(List *output_plugin_options,
ctx->reorder->begin = begin_cb_wrapper;
ctx->reorder->apply_change = change_cb_wrapper;
ctx->reorder->commit = commit_cb_wrapper;
+ ctx->reorder->message = message_cb_wrapper;
ctx->out = makeStringInfo();
ctx->prepare_write = prepare_write;
@@ -702,6 +706,40 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
return ret;
}
+static void
+message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn, bool transactional,
+ const char *prefix, Size message_size, const char *message)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ if (ctx->callbacks.message_cb == NULL)
+ return;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback_name = "message";
+ state.report_location = message_lsn;
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* set output state */
+ ctx->accept_writes = true;
+ ctx->write_xid = txn != NULL ? txn->xid : InvalidTransactionId;
+ ctx->write_location = message_lsn;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.message_cb(ctx, txn, message_lsn, transactional, prefix,
+ message_size, message);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
/*
* Set the required catalog xmin horizon for historic snapshots in the current
* replication slot.
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index dd6cd62..69d2000 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -24,6 +24,8 @@
#include "access/xlog_internal.h"
#include "access/xlogutils.h"
+#include "access/xact.h"
+
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -41,6 +43,7 @@
#include "replication/decode.h"
#include "replication/logical.h"
#include "replication/logicalfuncs.h"
+#include "replication/message.h"
#include "storage/fd.h"
@@ -380,3 +383,27 @@ pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS)
{
return pg_logical_slot_get_changes_guts(fcinfo, false, true);
}
+
+
+/*
+ * SQL function for writing logical decding message into WAL.
+ */
+Datum
+pg_logical_emit_message_bytea(PG_FUNCTION_ARGS)
+{
+ bool transactional = PG_GETARG_BOOL(0);
+ char *prefix = text_to_cstring(PG_GETARG_TEXT_PP(1));
+ bytea *data = PG_GETARG_BYTEA_PP(2);
+ XLogRecPtr lsn;
+
+ lsn = LogLogicalMessage(prefix, VARDATA_ANY(data), VARSIZE_ANY_EXHDR(data),
+ transactional);
+ PG_RETURN_LSN(lsn);
+}
+
+Datum
+pg_logical_emit_message_text(PG_FUNCTION_ARGS)
+{
+ /* bytea and text are compatible */
+ return pg_logical_emit_message_bytea(fcinfo);
+}
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
new file mode 100644
index 0000000..684f799
--- /dev/null
+++ b/src/backend/replication/logical/message.c
@@ -0,0 +1,87 @@
+/*-------------------------------------------------------------------------
+ *
+ * message.c
+ * Generic logical messages.
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/message.c
+ *
+ * NOTES
+ *
+ * Generic logical messages allow XLOG logging of arbitrary binary blobs that
+ * get passed to the logical decoding plugin. In normal XLOG processing they
+ * are same as NOOP.
+ *
+ * These messages can be either transactional or non-transactional.
+ * Transactional messages are part of current transaction and will be sent to
+ * decoding plugin using in a same way as DML operations.
+ * Non-transactional messages are sent to the plugin at the time when the
+ * logical decoding reads them from XLOG. This also means that transactional
+ * messages won't be delivered if the transaction was rolled back but the
+ * non-transactional one will be delivered always.
+ *
+ * Every message carries prefix to avoid conflicts between different decoding
+ * plugins. The plugin authors must take extra care to use unique prefix,
+ * good options seems to be for example to use the name of the extension.
+ *
+ * ---------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+
+#include "catalog/indexing.h"
+
+#include "nodes/execnodes.h"
+
+#include "replication/message.h"
+#include "replication/logical.h"
+
+#include "utils/memutils.h"
+
+/*
+ * Write logical decoding message into XLog.
+ */
+XLogRecPtr
+LogLogicalMessage(const char *prefix, const char *message, size_t size,
+ bool transactional)
+{
+ xl_logical_message xlrec;
+
+ /*
+ * Force xid to be allocated if we're emitting a transactional message.
+ */
+ if (transactional)
+ {
+ Assert(IsTransactionState());
+ GetCurrentTransactionId();
+ }
+
+ xlrec.transactional = transactional;
+ xlrec.prefix_size = strlen(prefix) + 1;
+ xlrec.message_size = size;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) &xlrec, SizeOfLogicalMessage);
+ XLogRegisterData((char *) prefix, xlrec.prefix_size);
+ XLogRegisterData((char *) message, size);
+
+ return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
+}
+
+/*
+ * Redo is basically just noop for logical decoding messages.
+ */
+void
+logicalmsg_redo(XLogReaderState *record)
+{
+ uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
+
+ if (info != XLOG_LOGICAL_MESSAGE)
+ elog(PANIC, "logicalmsg_redo: unknown op code %u", info);
+
+ /* This is only interesting for logical decoding, see decode.c. */
+}
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 9d78c8c..52c6986 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -414,6 +414,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
change->data.tp.oldtuple = NULL;
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ if (change->data.msg.prefix != NULL)
+ pfree(change->data.msg.prefix);
+ change->data.msg.prefix = NULL;
+ if (change->data.msg.message != NULL)
+ pfree(change->data.msg.message);
+ change->data.msg.message = NULL;
+ break;
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
if (change->data.snapshot)
{
@@ -627,6 +635,61 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
ReorderBufferCheckSerializeTXN(rb, txn);
}
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
+ Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message)
+{
+ if (transactional)
+ {
+ MemoryContext oldcontext;
+ ReorderBufferChange *change;
+
+ Assert(xid != InvalidTransactionId);
+
+ oldcontext = MemoryContextSwitchTo(rb->context);
+
+ change = ReorderBufferGetChange(rb);
+ change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+ change->data.msg.prefix = pstrdup(prefix);
+ change->data.msg.message_size = message_size;
+ change->data.msg.message = palloc(message_size);
+ memcpy(change->data.msg.message, message, message_size);
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+
+ MemoryContextSwitchTo(oldcontext);
+ }
+ else
+ {
+ ReorderBufferTXN *txn = NULL;
+ volatile Snapshot snapshot_now = snapshot;
+
+ if (xid != InvalidTransactionId)
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ /* setup snapshot to allow catalog access */
+ SetupHistoricSnapshot(snapshot_now, NULL);
+ PG_TRY();
+ {
+ rb->message(rb, txn, lsn, false, prefix, message_size, message);
+
+ TeardownHistoricSnapshot(false);
+ }
+ PG_CATCH();
+ {
+ TeardownHistoricSnapshot(true);
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+ }
+}
+
+
static void
AssertTXNLsnOrder(ReorderBuffer *rb)
{
@@ -1493,6 +1556,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
specinsert = change;
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ rb->message(rb, txn, change->lsn, true,
+ change->data.msg.prefix,
+ change->data.msg.message_size,
+ change->data.msg.message);
+ break;
+
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
/* get rid of the old */
TeardownHistoricSnapshot(false);
@@ -2159,6 +2229,33 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
}
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ char *data;
+ Size prefix_size = strlen(change->data.msg.prefix) + 1;
+
+ sz += prefix_size + change->data.msg.message_size +
+ sizeof(Size) + sizeof(Size);
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+
+ /* write the prefix including the size */
+ memcpy(data, &prefix_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.prefix,
+ prefix_size);
+ data += prefix_size;
+
+ /* write the message including the size */
+ memcpy(data, &change->data.msg.message_size, sizeof(Size));
+ data += sizeof(Size);
+ memcpy(data, change->data.msg.message,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot snap;
@@ -2415,6 +2512,30 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
}
break;
+ case REORDER_BUFFER_CHANGE_MESSAGE:
+ {
+ Size prefix_size;
+
+ /* read prefix */
+ memcpy(&prefix_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.prefix = MemoryContextAlloc(rb->context,
+ prefix_size);
+ memcpy(change->data.msg.prefix, data, prefix_size);
+ Assert(change->data.msg.prefix[prefix_size-1] == '\0');
+ data += prefix_size;
+
+ /* read the messsage */
+ memcpy(&change->data.msg.message_size, data, sizeof(Size));
+ data += sizeof(Size);
+ change->data.msg.message = MemoryContextAlloc(rb->context,
+ change->data.msg.message_size);
+ memcpy(change->data.msg.message, data,
+ change->data.msg.message_size);
+ data += change->data.msg.message_size;
+
+ break;
+ }
case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
{
Snapshot oldsnap;
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 179b85a..b4dc617 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -605,6 +605,25 @@ SnapBuildExportSnapshot(SnapBuild *builder)
}
/*
+ * Ensure there is a snapshot and if not build one for current transaction.
+ */
+Snapshot
+SnapBuildGetOrBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ return builder->snapshot;
+}
+
+/*
* Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
* any. Aborts the previously started transaction and resets the resource
* owner back to its original value.
diff --git a/src/bin/pg_xlogdump/.gitignore b/src/bin/pg_xlogdump/.gitignore
index 33a1acf..c4783f1 100644
--- a/src/bin/pg_xlogdump/.gitignore
+++ b/src/bin/pg_xlogdump/.gitignore
@@ -1,23 +1,4 @@
/pg_xlogdump
# Source files copied from src/backend/access/rmgrdesc/
-/brindesc.c
-/clogdesc.c
-/committsdesc.c
-/dbasedesc.c
-/genericdesc.c
-/gindesc.c
-/gistdesc.c
-/hashdesc.c
-/heapdesc.c
-/mxactdesc.c
-/nbtdesc.c
-/relmapdesc.c
-/replorigindesc.c
-/seqdesc.c
-/smgrdesc.c
-/spgdesc.c
-/standbydesc.c
-/tblspcdesc.c
-/xactdesc.c
-/xlogdesc.c
+/*desc.c
/xlogreader.c
diff --git a/src/bin/pg_xlogdump/rmgrdesc.c b/src/bin/pg_xlogdump/rmgrdesc.c
index cff7e59..017b9c5 100644
--- a/src/bin/pg_xlogdump/rmgrdesc.c
+++ b/src/bin/pg_xlogdump/rmgrdesc.c
@@ -26,6 +26,7 @@
#include "commands/dbcommands_xlog.h"
#include "commands/sequence.h"
#include "commands/tablespace.h"
+#include "replication/message.h"
#include "replication/origin.h"
#include "rmgrdesc.h"
#include "storage/standbydefs.h"
diff --git a/src/include/access/rmgrlist.h b/src/include/access/rmgrlist.h
index 3cfe6f7..a7a0ae2 100644
--- a/src/include/access/rmgrlist.h
+++ b/src/include/access/rmgrlist.h
@@ -46,3 +46,4 @@ PG_RMGR(RM_BRIN_ID, "BRIN", brin_redo, brin_desc, brin_identify, NULL, NULL)
PG_RMGR(RM_COMMIT_TS_ID, "CommitTs", commit_ts_redo, commit_ts_desc, commit_ts_identify, NULL, NULL)
PG_RMGR(RM_REPLORIGIN_ID, "ReplicationOrigin", replorigin_redo, replorigin_desc, replorigin_identify, NULL, NULL)
PG_RMGR(RM_GENERIC_ID, "Generic", generic_redo, generic_desc, generic_identify, NULL, NULL)
+PG_RMGR(RM_LOGICALMSG_ID, "LogicalMessage", logicalmsg_redo, logicalmsg_desc, logicalmsg_identify, NULL, NULL)
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index c86b920..e5d20f0 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5125,6 +5125,10 @@ DATA(insert OID = 3784 ( pg_logical_slot_peek_changes PGNSP PGUID 12 1000 1000
DESCR("peek at changes from replication slot");
DATA(insert OID = 3785 ( pg_logical_slot_peek_binary_changes PGNSP PGUID 12 1000 1000 25 0 f f f f f t v u 4 0 2249 "19 3220 23 1009" "{19,3220,23,1009,3220,28,17}" "{i,i,i,v,o,o,o}" "{slot_name,upto_lsn,upto_nchanges,options,location,xid,data}" _null_ _null_ pg_logical_slot_peek_binary_changes _null_ _null_ _null_ ));
DESCR("peek at binary changes from replication slot");
+DATA(insert OID = 3577 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 25" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_text _null_ _null_ _null_ ));
+DESCR("emit a textual logical decoding message");
+DATA(insert OID = 3578 ( pg_logical_emit_message PGNSP PGUID 12 1 0 0 0 f f f f t f v u 3 0 3220 "16 25 17" _null_ _null_ _null_ _null_ _null_ pg_logical_emit_message_bytea _null_ _null_ _null_ ));
+DESCR("emit a binary logical decoding message");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s s 0 0 2249 "" "{26,26,23,16,16,16,25,25,25,25,1009,1009}" "{o,o,o,o,o,o,o,o,o,o,o,o}" "{classid, objid, objsubid, original, normal, is_temporary, object_type, schema_name, object_name, object_identity, address_names, address_args}" _null_ _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
diff --git a/src/include/replication/logicalfuncs.h b/src/include/replication/logicalfuncs.h
index c87a1df..5540414 100644
--- a/src/include/replication/logicalfuncs.h
+++ b/src/include/replication/logicalfuncs.h
@@ -21,4 +21,6 @@ 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);
+extern Datum pg_logical_emit_message_bytea(PG_FUNCTION_ARGS);
+extern Datum pg_logical_emit_message_text(PG_FUNCTION_ARGS);
#endif
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
new file mode 100644
index 0000000..8b968d5
--- /dev/null
+++ b/src/include/replication/message.h
@@ -0,0 +1,41 @@
+/*-------------------------------------------------------------------------
+ * message.h
+ * Exports from replication/logical/message.c
+ *
+ * Copyright (c) 2013-2016, PostgreSQL Global Development Group
+ *
+ * src/include/replication/message.h
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_LOGICAL_MESSAGE_H
+#define PG_LOGICAL_MESSAGE_H
+
+#include "access/xlog.h"
+#include "access/xlogdefs.h"
+#include "access/xlogreader.h"
+
+/*
+ * Generic logical decoding message wal record.
+ */
+typedef struct xl_logical_message
+{
+ bool transactional; /* is message transactional? */
+ Size prefix_size; /* length of prefix */
+ Size message_size; /* size of the message */
+ char message[FLEXIBLE_ARRAY_MEMBER]; /* message including the null
+ * terminated prefix of length
+ * prefix_size */
+} xl_logical_message;
+
+#define SizeOfLogicalMessage (offsetof(xl_logical_message, message))
+
+extern XLogRecPtr LogLogicalMessage(const char *prefix, const char *message,
+ size_t size, bool transactional);
+
+/* RMGR API*/
+#define XLOG_LOGICAL_MESSAGE 0x00
+void logicalmsg_redo(XLogReaderState *record);
+void logicalmsg_desc(StringInfo buf, XLogReaderState *record);
+const char *logicalmsg_identify(uint8 info);
+
+#endif /* PG_LOGICAL_MESSAGE_H */
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 577b12e..3a2ca98 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -74,6 +74,18 @@ typedef void (*LogicalDecodeCommitCB) (
XLogRecPtr commit_lsn);
/*
+ * Called for the generic logical decoding messages.
+ */
+typedef void (*LogicalDecodeMessageCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix,
+ Size message_size,
+ const char *message);
+
+/*
* Filter changes by origin.
*/
typedef bool (*LogicalDecodeFilterByOriginCB) (
@@ -96,6 +108,7 @@ typedef struct OutputPluginCallbacks
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeMessageCB message_cb;
LogicalDecodeFilterByOriginCB filter_by_origin_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b52d06a..4c54953 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -54,6 +54,7 @@ enum ReorderBufferChangeType
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE,
+ REORDER_BUFFER_CHANGE_MESSAGE,
REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID,
@@ -98,6 +99,14 @@ typedef struct ReorderBufferChange
ReorderBufferTupleBuf *newtuple;
} tp;
+ /* Message with arbitrary data. */
+ struct
+ {
+ char *prefix;
+ Size message_size;
+ char *message;
+ } msg;
+
/* New snapshot, set when action == *_INTERNAL_SNAPSHOT */
Snapshot snapshot;
@@ -274,6 +283,15 @@ typedef void (*ReorderBufferCommitCB) (
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
+/* message callback signature */
+typedef void (*ReorderBufferMessageCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr message_lsn,
+ bool transactional,
+ const char *prefix, Size sz,
+ const char *message);
+
struct ReorderBuffer
{
/*
@@ -300,6 +318,7 @@ struct ReorderBuffer
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
+ ReorderBufferMessageCB message;
/*
* Pointer that will be passed untouched to the callbacks.
@@ -350,6 +369,9 @@ ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferQueueMessage(ReorderBuffer *, TransactionId, Snapshot snapshot, XLogRecPtr lsn,
+ bool transactional, const char *prefix,
+ Size message_size, const char *message);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time, RepOriginId origin_id, XLogRecPtr origin_lsn);
diff --git a/src/include/replication/snapbuild.h b/src/include/replication/snapbuild.h
index 75955af..c4127a1 100644
--- a/src/include/replication/snapbuild.h
+++ b/src/include/replication/snapbuild.h
@@ -63,6 +63,8 @@ extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+extern Snapshot SnapBuildGetOrBuildSnapshot(SnapBuild *builder,
+ TransactionId xid);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
--
1.9.1
On 4 April 2016 at 05:23, Petr Jelinek <petr@2ndquadrant.com> wrote:
I rebased this patch on top of current master as the generic wal commit
added some conflicting changes. Also fixed couple of typos in comments and
added non ascii message to test.
This looks good to me, so have marked it Ready For Committer.
I marked myself as Committer to show there was interest in this. If anyone
else would like to commit it, I am happy for you to do so.
--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/>
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
Committed. https://commitfest.postgresql.org/9/468/
Buildfarm error:
/messages/by-id/CAB7nPqROd2MXqy_5+cZJVhW0wHrrz6P8jV_RSbLcrXRTwLh7tQ@mail.gmail.com
Interesting issue. Mainly because the "ť" char it complains about
(utf-8 0xc5 0xa5) is accepted in the SELECT that generates the record. If
it's valid input it should be valid output, right? We didn't change the
client_encoding in the mean time. It makes sense though:
initdb on that animal says:
The database cluster will be initialized with locale "English_United
States.1252".
The default database encoding has accordingly been set to "WIN1252".
The regress script in question sets:
SET client_encoding = 'utf8';
but we're apparently round-tripping the data through the database encoding
at some point, then converting back to client_encoding for output.
Presumably that's when we're forming the text 'data' column in the
tuplestore produced by the get changes function, which will be in the
database encoding.
So setting client_encoding is not sufficient to make this work and the
non-7-bit-ascii part should be removed from the test, since it's not
allowed on all machines.
In some ways it seems like the argument to pg_logical_emit_message(...) should
be 'bytea'. That'd be much more convenient for application use. But then
it's a pain when using it via the text-format SQL interface calls, where
we've got no sensible way to output it.
--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Craig Ringer <craig@2ndquadrant.com> writes:
Interesting issue. Mainly because the "ť" char it complains about
(utf-8 0xc5 0xa5) is accepted in the SELECT that generates the record.
Uh, no, actually it's the SELECT that's failing.
The regress script in question sets:
SET client_encoding = 'utf8';
but we're apparently round-tripping the data through the database encoding
at some point, then converting back to client_encoding for output.
The conversion to DB encoding will happen the instant the query string
reaches the database. You can set client_encoding to whatever you want,
but the only characters that can appear in queries are those that exist
in both the client encoding and the database encoding.
In some ways it seems like the argument to pg_logical_emit_message(...) should
be 'bytea'. That'd be much more convenient for application use. But then
it's a pain when using it via the text-format SQL interface calls, where
we've got no sensible way to output it.
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database different from
the one where a change was originated? What's going to happen if a string
in WAL contains characters unrepresentable in that database? Do we even
have logic in there that will attempt to perform the necessary conversion?
And it is *necessary*, not optional, if you are going to claim that the
output of pg_logical_slot_get_changes is of type text.
regards, tom lane
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2016-04-06 10:15:59 -0400, Tom Lane wrote:
In some ways it seems like the argument to pg_logical_emit_message(...) should
be 'bytea'. That'd be much more convenient for application use. But then
it's a pain when using it via the text-format SQL interface calls, where
we've got no sensible way to output it.
There's a bytea version.
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database different from
the one where a change was originated?
You can execute it, but you'll get an error:
if (slot->data.database != MyDatabaseId)
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
(errmsg("replication slot \"%s\" was not created in this database",
NameStr(slot->data.name)))));
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Andres Freund <andres@anarazel.de> writes:
On 2016-04-06 10:15:59 -0400, Tom Lane wrote:
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database different from
the one where a change was originated?
You can execute it, but you'll get an error:
Oh good. I was afraid we had an unrecognized can o' worms here.
regards, tom lane
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2016-04-06 16:20:29 +0200, Andres Freund wrote:
On 2016-04-06 10:15:59 -0400, Tom Lane wrote:
In some ways it seems like the argument to pg_logical_emit_message(...) should
be 'bytea'. That'd be much more convenient for application use. But then
it's a pain when using it via the text-format SQL interface calls, where
we've got no sensible way to output it.There's a bytea version.
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database different from
the one where a change was originated?You can execute it, but you'll get an error:
if (slot->data.database != MyDatabaseId)
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
(errmsg("replication slot \"%s\" was not created in this database",
NameStr(slot->data.name)))));
Or so I thought. A look at the code shows a lack of database filtering
in DecodeLogicalMsgOp(). I think it also misses a FilterByOrigin()
check.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2016-04-06 10:24:52 -0400, Tom Lane wrote:
Andres Freund <andres@anarazel.de> writes:
On 2016-04-06 10:15:59 -0400, Tom Lane wrote:
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database different from
the one where a change was originated?You can execute it, but you'll get an error:
Oh good. I was afraid we had an unrecognized can o' worms here.
As posted nearby, there's a hole in that defense; for the messages
only. Pretty easy to solve though.
Allowing logical decoding from a difference would have a lot of
problems; primarily we couldn't actually look up any catalog state. But
even leaving that aside, it'd end up being pretty hard to interpret
database contents without knowledge about encoding.
Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 6 April 2016 at 15:29, Andres Freund <andres@anarazel.de> wrote:
On 2016-04-06 10:24:52 -0400, Tom Lane wrote:
Andres Freund <andres@anarazel.de> writes:
On 2016-04-06 10:15:59 -0400, Tom Lane wrote:
Well, that's something worth thinking about. I assume that
pg_logical_slot_get_changes could be executed in a database differentfrom
the one where a change was originated?
You can execute it, but you'll get an error:
Oh good. I was afraid we had an unrecognized can o' worms here.
As posted nearby, there's a hole in that defense; for the messages
only. Pretty easy to solve though.
My instinct was to put in a test for non-ascii text; even if we can't keep
that test, it has highlighted a hole we wouldn't have spotted for a while,
so I'll call that "good catch" then.
Perhaps easy to solve, but how do we test it is solved?
--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/>
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
On 2016-04-06 16:49:17 +0100, Simon Riggs wrote:
Perhaps easy to solve, but how do we test it is solved?
Maybe something like
-- drain
pg_logical_slot_get_changes(...);
-- generate message in different database, to ensure it's not processed
-- in this database
\c template1
SELECT pg_logical_emit_message(...);
\c postgres
-- check
pg_logical_slot_get_changes(..);
It's a bit ugly to hardcode database names :/
Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 06/04/16 17:55, Andres Freund wrote:
On 2016-04-06 16:49:17 +0100, Simon Riggs wrote:
Perhaps easy to solve, but how do we test it is solved?
Maybe something like
-- drain
pg_logical_slot_get_changes(...);
-- generate message in different database, to ensure it's not processed
-- in this database
\c template1
SELECT pg_logical_emit_message(...);
\c postgres
-- check
pg_logical_slot_get_changes(..);It's a bit ugly to hardcode database names :/
Attached patch adds filtering of both database and origin. Added tests
with slightly less hardcoding than what you proposed above.
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-db-origin-filter-fix.patchtext/x-diff; name=logical-messages-db-origin-filter-fix.patchDownload
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
index 70130e9..a5b13c5 100644
--- a/contrib/test_decoding/expected/messages.out
+++ b/contrib/test_decoding/expected/messages.out
@@ -1,6 +1,5 @@
-- predictability
SET synchronous_commit = on;
-SET client_encoding = 'utf8';
SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
?column?
----------
@@ -71,9 +70,32 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'for
message: transactional: 1 prefix: test, sz: 11 content:czechtastic
(7 rows)
-SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+-- test db filtering
+CREATE DATABASE test_logical_messages;
+\set prevdb :DBNAME
+\c test_logical_messages
+SELECT 'otherdb1' FROM pg_logical_emit_message(false, 'test', 'otherdb1');
?column?
----------
- init
+ otherdb1
+(1 row)
+
+SELECT 'otherdb2' FROM pg_logical_emit_message(true, 'test', 'otherdb2');
+ ?column?
+----------
+ otherdb2
+(1 row)
+
+\c :prevdb
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------
+(0 rows)
+
+DROP DATABASE test_logical_messages;
+SELECT 'cleanup' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ cleanup
(1 row)
diff --git a/contrib/test_decoding/expected/replorigin.out b/contrib/test_decoding/expected/replorigin.out
index c0f5125..8e8889d 100644
--- a/contrib/test_decoding/expected/replorigin.out
+++ b/contrib/test_decoding/expected/replorigin.out
@@ -59,6 +59,12 @@ SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
-- ensure we prevent duplicate setup
SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
ERROR: cannot setup replication origin when one is already setup
+SELECT 'this message will not be decoded' FROM pg_logical_emit_message(false, 'test', 'this message will not be decoded');
+ ?column?
+----------------------------------
+ this message will not be decoded
+(1 row)
+
BEGIN;
-- setup transaction origin
SELECT pg_replication_origin_xact_setup('0/aabbccdd', '2013-01-01 00:00');
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
index 4aedb04..eba7d7a 100644
--- a/contrib/test_decoding/sql/messages.sql
+++ b/contrib/test_decoding/sql/messages.sql
@@ -1,6 +1,5 @@
-- predictability
SET synchronous_commit = on;
-SET client_encoding = 'utf8';
SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
@@ -22,4 +21,17 @@ SELECT 'ignorethis' FROM pg_logical_emit_message(true, 'test', 'czechtastic');
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
-SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+-- test db filtering
+CREATE DATABASE test_logical_messages;
+\set prevdb :DBNAME
+\c test_logical_messages
+
+SELECT 'otherdb1' FROM pg_logical_emit_message(false, 'test', 'otherdb1');
+SELECT 'otherdb2' FROM pg_logical_emit_message(true, 'test', 'otherdb2');
+
+\c :prevdb
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+DROP DATABASE test_logical_messages;
+
+SELECT 'cleanup' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/sql/replorigin.sql b/contrib/test_decoding/sql/replorigin.sql
index e12404e..a33e460bb 100644
--- a/contrib/test_decoding/sql/replorigin.sql
+++ b/contrib/test_decoding/sql/replorigin.sql
@@ -31,6 +31,8 @@ SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
-- ensure we prevent duplicate setup
SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
+SELECT 'this message will not be decoded' FROM pg_logical_emit_message(false, 'test', 'this message will not be decoded');
+
BEGIN;
-- setup transaction origin
SELECT pg_replication_origin_xact_setup('0/aabbccdd', '2013-01-01 00:00');
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 3e80c4a..0cdb0b8 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -464,6 +464,15 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+static inline bool
+FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
+{
+ if (ctx->callbacks.filter_by_origin_cb == NULL)
+ return false;
+
+ return filter_by_origin_cb_wrapper(ctx, origin_id);
+}
+
/*
* Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
*/
@@ -474,6 +483,7 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
XLogReaderState *r = buf->record;
TransactionId xid = XLogRecGetXid(r);
uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ RepOriginId origin_id = XLogRecGetOrigin(r);
Snapshot snapshot;
xl_logical_message *message;
@@ -488,6 +498,10 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
message = (xl_logical_message *) XLogRecGetData(r);
+ if (message->dbId != ctx->slot->data.database ||
+ FilterByOrigin(ctx, origin_id))
+ return;
+
if (message->transactional &&
!SnapBuildProcessChange(builder, xid, buf->origptr))
return;
@@ -504,15 +518,6 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
message->message + message->prefix_size);
}
-static inline bool
-FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
-{
- if (ctx->callbacks.filter_by_origin_cb == NULL)
- return false;
-
- return filter_by_origin_cb_wrapper(ctx, origin_id);
-}
-
/*
* Consolidated commit record handling between the different form of commit
* records.
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
index 684f799..efcc25a 100644
--- a/src/backend/replication/logical/message.c
+++ b/src/backend/replication/logical/message.c
@@ -31,6 +31,8 @@
#include "postgres.h"
+#include "miscadmin.h"
+
#include "access/xact.h"
#include "catalog/indexing.h"
@@ -60,6 +62,7 @@ LogLogicalMessage(const char *prefix, const char *message, size_t size,
GetCurrentTransactionId();
}
+ xlrec.dbId = MyDatabaseId;
xlrec.transactional = transactional;
xlrec.prefix_size = strlen(prefix) + 1;
xlrec.message_size = size;
@@ -69,6 +72,9 @@ LogLogicalMessage(const char *prefix, const char *message, size_t size,
XLogRegisterData((char *) prefix, xlrec.prefix_size);
XLogRegisterData((char *) message, size);
+ /* allow origin filtering */
+ XLogIncludeOrigin();
+
return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
}
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
index 8b968d5..c9fd477 100644
--- a/src/include/replication/message.h
+++ b/src/include/replication/message.h
@@ -19,6 +19,7 @@
*/
typedef struct xl_logical_message
{
+ Oid dbId; /* Oid of the database */
bool transactional; /* is message transactional? */
Size prefix_size; /* length of prefix */
Size message_size; /* size of the message */
On Thu, Apr 7, 2016 at 12:55 AM, Andres Freund <andres@anarazel.de> wrote:
On 2016-04-06 16:49:17 +0100, Simon Riggs wrote:
Perhaps easy to solve, but how do we test it is solved?
Maybe something like
-- drain
pg_logical_slot_get_changes(...);
-- generate message in different database, to ensure it's not processed
-- in this database
\c template1
SELECT pg_logical_emit_message(...);
\c postgres
-- check
pg_logical_slot_get_changes(..);It's a bit ugly to hardcode database names :/
When running installcheck, there is no way to be sure that databases
template1 and/or postgres exist on a server, so this test would fail
because of that.
--
Michael
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On April 7, 2016 2:26:41 AM GMT+02:00, Michael Paquier <michael.paquier@gmail.com> wrote:
On Thu, Apr 7, 2016 at 12:55 AM, Andres Freund <andres@anarazel.de>
wrote:On 2016-04-06 16:49:17 +0100, Simon Riggs wrote:
Perhaps easy to solve, but how do we test it is solved?
Maybe something like
-- drain
pg_logical_slot_get_changes(...);
-- generate message in different database, to ensure it's notprocessed
-- in this database
\c template1
SELECT pg_logical_emit_message(...);
\c postgres
-- check
pg_logical_slot_get_changes(..);It's a bit ugly to hardcode database names :/
When running installcheck, there is no way to be sure that databases
template1 and/or postgres exist on a server, so this test would fail
because of that.
No need to hardcode postgres, see Petr's reply. I'm not concerned about template 1 not being there -if you tinkered with things in that level it's unlikely that tests will succeed. Also, remember, this is in a test cluster created by the regression script, and there's no installcheck support anyway (because of the required settings for logical decoding) anyway
--
Sent from my Android device with K-9 Mail. Please excuse my brevity.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
On 2016-04-06 20:03:20 +0200, Petr Jelinek wrote:
Attached patch adds filtering of both database and origin. Added tests with
slightly less hardcoding than what you proposed above.
Not a fan of creating & dropping another database - that's really rather
expensive. And we're in a target that doesn't support installcheck, so
relying on template1's existance seems fine...
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out index 70130e9..a5b13c5 100644 --- a/contrib/test_decoding/expected/messages.out +++ b/contrib/test_decoding/expected/messages.out @@ -1,6 +1,5 @@ -- predictability SET synchronous_commit = on; -SET client_encoding = 'utf8';
I guess that's just from the previous test with czech text?
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 07/04/16 12:26, Andres Freund wrote:
Hi,
On 2016-04-06 20:03:20 +0200, Petr Jelinek wrote:
Attached patch adds filtering of both database and origin. Added tests with
slightly less hardcoding than what you proposed above.Not a fan of creating & dropping another database - that's really rather
expensive. And we're in a target that doesn't support installcheck, so
relying on template1's existance seems fine...
Makes sense, changed that bit.
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out index 70130e9..a5b13c5 100644 --- a/contrib/test_decoding/expected/messages.out +++ b/contrib/test_decoding/expected/messages.out @@ -1,6 +1,5 @@ -- predictability SET synchronous_commit = on; -SET client_encoding = 'utf8';I guess that's just from the previous test with czech text?
Yeah it's cleanup after the d25379eb23383f1d2f969e65e0332b47c19aea94 .
--
Petr Jelinek http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
Attachments:
logical-messages-db-origin-filter-fix2.patchtext/x-diff; name=logical-messages-db-origin-filter-fix2.patchDownload
diff --git a/contrib/test_decoding/expected/messages.out b/contrib/test_decoding/expected/messages.out
index 70130e9..c75d401 100644
--- a/contrib/test_decoding/expected/messages.out
+++ b/contrib/test_decoding/expected/messages.out
@@ -1,6 +1,5 @@
-- predictability
SET synchronous_commit = on;
-SET client_encoding = 'utf8';
SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
?column?
----------
@@ -71,9 +70,30 @@ SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'for
message: transactional: 1 prefix: test, sz: 11 content:czechtastic
(7 rows)
-SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+-- test db filtering
+\set prevdb :DBNAME
+\c template1
+SELECT 'otherdb1' FROM pg_logical_emit_message(false, 'test', 'otherdb1');
?column?
----------
- init
+ otherdb1
+(1 row)
+
+SELECT 'otherdb2' FROM pg_logical_emit_message(true, 'test', 'otherdb2');
+ ?column?
+----------
+ otherdb2
+(1 row)
+
+\c :prevdb
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+ data
+------
+(0 rows)
+
+SELECT 'cleanup' FROM pg_drop_replication_slot('regression_slot');
+ ?column?
+----------
+ cleanup
(1 row)
diff --git a/contrib/test_decoding/expected/replorigin.out b/contrib/test_decoding/expected/replorigin.out
index c0f5125..8e8889d 100644
--- a/contrib/test_decoding/expected/replorigin.out
+++ b/contrib/test_decoding/expected/replorigin.out
@@ -59,6 +59,12 @@ SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
-- ensure we prevent duplicate setup
SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
ERROR: cannot setup replication origin when one is already setup
+SELECT 'this message will not be decoded' FROM pg_logical_emit_message(false, 'test', 'this message will not be decoded');
+ ?column?
+----------------------------------
+ this message will not be decoded
+(1 row)
+
BEGIN;
-- setup transaction origin
SELECT pg_replication_origin_xact_setup('0/aabbccdd', '2013-01-01 00:00');
diff --git a/contrib/test_decoding/sql/messages.sql b/contrib/test_decoding/sql/messages.sql
index 4aedb04..cf3f773 100644
--- a/contrib/test_decoding/sql/messages.sql
+++ b/contrib/test_decoding/sql/messages.sql
@@ -1,6 +1,5 @@
-- predictability
SET synchronous_commit = on;
-SET client_encoding = 'utf8';
SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
@@ -22,4 +21,14 @@ SELECT 'ignorethis' FROM pg_logical_emit_message(true, 'test', 'czechtastic');
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
-SELECT 'init' FROM pg_drop_replication_slot('regression_slot');
+-- test db filtering
+\set prevdb :DBNAME
+\c template1
+
+SELECT 'otherdb1' FROM pg_logical_emit_message(false, 'test', 'otherdb1');
+SELECT 'otherdb2' FROM pg_logical_emit_message(true, 'test', 'otherdb2');
+
+\c :prevdb
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'force-binary', '0', 'skip-empty-xacts', '1');
+
+SELECT 'cleanup' FROM pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/sql/replorigin.sql b/contrib/test_decoding/sql/replorigin.sql
index e12404e..a33e460bb 100644
--- a/contrib/test_decoding/sql/replorigin.sql
+++ b/contrib/test_decoding/sql/replorigin.sql
@@ -31,6 +31,8 @@ SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
-- ensure we prevent duplicate setup
SELECT pg_replication_origin_session_setup('test_decoding: regression_slot');
+SELECT 'this message will not be decoded' FROM pg_logical_emit_message(false, 'test', 'this message will not be decoded');
+
BEGIN;
-- setup transaction origin
SELECT pg_replication_origin_xact_setup('0/aabbccdd', '2013-01-01 00:00');
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 3e80c4a..0cdb0b8 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -464,6 +464,15 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
}
}
+static inline bool
+FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
+{
+ if (ctx->callbacks.filter_by_origin_cb == NULL)
+ return false;
+
+ return filter_by_origin_cb_wrapper(ctx, origin_id);
+}
+
/*
* Handle rmgr LOGICALMSG_ID records for DecodeRecordIntoReorderBuffer().
*/
@@ -474,6 +483,7 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
XLogReaderState *r = buf->record;
TransactionId xid = XLogRecGetXid(r);
uint8 info = XLogRecGetInfo(r) & ~XLR_INFO_MASK;
+ RepOriginId origin_id = XLogRecGetOrigin(r);
Snapshot snapshot;
xl_logical_message *message;
@@ -488,6 +498,10 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
message = (xl_logical_message *) XLogRecGetData(r);
+ if (message->dbId != ctx->slot->data.database ||
+ FilterByOrigin(ctx, origin_id))
+ return;
+
if (message->transactional &&
!SnapBuildProcessChange(builder, xid, buf->origptr))
return;
@@ -504,15 +518,6 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
message->message + message->prefix_size);
}
-static inline bool
-FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
-{
- if (ctx->callbacks.filter_by_origin_cb == NULL)
- return false;
-
- return filter_by_origin_cb_wrapper(ctx, origin_id);
-}
-
/*
* Consolidated commit record handling between the different form of commit
* records.
diff --git a/src/backend/replication/logical/message.c b/src/backend/replication/logical/message.c
index 684f799..efcc25a 100644
--- a/src/backend/replication/logical/message.c
+++ b/src/backend/replication/logical/message.c
@@ -31,6 +31,8 @@
#include "postgres.h"
+#include "miscadmin.h"
+
#include "access/xact.h"
#include "catalog/indexing.h"
@@ -60,6 +62,7 @@ LogLogicalMessage(const char *prefix, const char *message, size_t size,
GetCurrentTransactionId();
}
+ xlrec.dbId = MyDatabaseId;
xlrec.transactional = transactional;
xlrec.prefix_size = strlen(prefix) + 1;
xlrec.message_size = size;
@@ -69,6 +72,9 @@ LogLogicalMessage(const char *prefix, const char *message, size_t size,
XLogRegisterData((char *) prefix, xlrec.prefix_size);
XLogRegisterData((char *) message, size);
+ /* allow origin filtering */
+ XLogIncludeOrigin();
+
return XLogInsert(RM_LOGICALMSG_ID, XLOG_LOGICAL_MESSAGE);
}
diff --git a/src/include/replication/message.h b/src/include/replication/message.h
index 8b968d5..c9fd477 100644
--- a/src/include/replication/message.h
+++ b/src/include/replication/message.h
@@ -19,6 +19,7 @@
*/
typedef struct xl_logical_message
{
+ Oid dbId; /* Oid of the database */
bool transactional; /* is message transactional? */
Size prefix_size; /* length of prefix */
Size message_size; /* size of the message */
On 2016-04-07 19:53:56 +0200, Petr Jelinek wrote:
On 07/04/16 12:26, Andres Freund wrote:
Hi,
On 2016-04-06 20:03:20 +0200, Petr Jelinek wrote:
Attached patch adds filtering of both database and origin. Added tests with
slightly less hardcoding than what you proposed above.Not a fan of creating & dropping another database - that's really rather
expensive. And we're in a target that doesn't support installcheck, so
relying on template1's existance seems fine...Makes sense, changed that bit.
I've pushed this. I also noticed that both this patch (that's ok,
committers commonly do that, but a reminder is nice) and the original
commit ommitted to bump XLOG_PAGE_MAGIC. The original commit also
omitted bumping catversion. btw.
Thanks for the patch.
Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers