This is an automated email from the ASF dual-hosted git repository.
timoninmaxim pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/master by this push:
new e9c413ba821 IGNITE-25774 Add docs for PostgreSQL cdc-extension (#12193)
e9c413ba821 is described below
commit e9c413ba821739bb115b78ace8076d56d83eea1b
Author: Maksim Davydov <[email protected]>
AuthorDate: Fri Aug 22 16:11:11 2025 +0300
IGNITE-25774 Add docs for PostgreSQL cdc-extension (#12193)
---
docs/_data/toc.yaml | 12 +-
.../change-data-capture/ignite-to-ignite.adoc | 75 +++++++
.../ignite-to-kafka.adoc} | 165 +-------------
.../change-data-capture/ignite-to-postgresql.adoc | 248 +++++++++++++++++++++
.../change-data-capture/overview.adoc | 119 ++++++++++
5 files changed, 453 insertions(+), 166 deletions(-)
diff --git a/docs/_data/toc.yaml b/docs/_data/toc.yaml
index 3b7ce272095..75cb35532d9 100644
--- a/docs/_data/toc.yaml
+++ b/docs/_data/toc.yaml
@@ -553,12 +553,20 @@
url: extensions-and-integrations/cassandra/usage-examples
- title: DDL Generator
url: extensions-and-integrations/cassandra/ddl-generator
+ - title: Change Data Capture
+ items:
+ - title: Overview
+ url: extensions-and-integrations/change-data-capture/overview
+ - title: Ignite to Ignite Replication
+ url: extensions-and-integrations/change-data-capture/ignite-to-ignite
+ - title: Ignite to Kafka Replication
+ url: extensions-and-integrations/change-data-capture/ignite-to-kafka
+ - title: Ignite to PostgreSQL Replication
+ url:
extensions-and-integrations/change-data-capture/ignite-to-postgresql
- title: PHP PDO
url: extensions-and-integrations/php-pdo
- title: Performance Statistics
url: extensions-and-integrations/performance-statistics
- - title: Change Data Capture
- url: extensions-and-integrations/change-data-capture-extensions
- title: ZooKeeper IP Finder
url: extensions-and-integrations/zookeeper-ip-finder-ext
- title: Plugins
diff --git
a/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-ignite.adoc
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-ignite.adoc
new file mode 100644
index 00000000000..2ee68c5f7d9
--- /dev/null
+++
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-ignite.adoc
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+= Ignite to Ignite cross-cluster replication
+
+== Ignite to Java Thin Client CDC streamer
+This streamer starts link:thin-clients/java-thin-client[Java Thin Client]
which connects to destination cluster.
+After connection is established, all changes captured by CDC will be
replicated to destination cluster.
+
+NOTE: Instances of `ignite-cdc.sh` with configured streamer should be started
on each server node of source cluster to capture all changes.
+
+image:../../assets/images/integrations/CDC-ignite2igniteClient.svg[]
+
+=== Configuration
+
+[cols="20%,45%,35%",opts="header"]
+|===
+|Name |Description | Default value
+| `caches` | Set of cache names to replicate. | null
+| `destinationClientConfiguration` | Client configuration of thin client that
will connect to destination cluster to replicate changes. | null
+| `onlyPrimary` | Flag to handle changes only on primary node. | `false`
+| `maxBatchSize` | Maximum number of events to be sent to destination cluster
in a single batch. | 1024
+|===
+
+=== Metrics
+
+[cols="25%,75%",opts="header"]
+|===
+|Name |Description
+| `EventsCount` | Count of messages applied to destination cluster.
+| `LastEventTime` | Timestamp of last applied event to destination cluster.
+| `TypesCount` | Count of binary types events applied to destination cluster.
+| `MappingsCount` | Count of mappings events applied to destination cluster
+|===
+
+== Ignite to Ignite CDC streamer
+This streamer starts client node which connects to destination cluster.
+After connection is established, all changes captured by CDC will be
replicated to destination cluster.
+
+NOTE: Instances of `ignite-cdc.sh` with configured streamer should be started
on each server node of source cluster to capture all changes.
+
+image:../../assets/images/integrations/CDC-ignite2ignite.svg[]
+
+=== Configuration
+
+[cols="20%,45%,35%",opts="header"]
+|===
+|Name |Description | Default value
+| `caches` | Set of cache names to replicate. | null
+| `destinationIgniteConfiguration` | Ignite configuration of client nodes that
will connect to destination cluster to replicate changes. | null
+| `onlyPrimary` | Flag to handle changes only on primary node. | `false`
+| `maxBatchSize` | Maximum number of events to be sent to destination cluster
in a single batch. | 1024
+|===
+
+=== Metrics
+
+[cols="25%,75%",opts="header"]
+|===
+|Name |Description
+| `EventsCount` | Count of messages applied to destination cluster.
+| `LastEventTime` | Timestamp of last applied event to destination cluster.
+| `TypesCount` | Count of binary types events applied to destination cluster.
+| `MappingsCount` | Count of mappings events applied to destination cluster
+|===
diff --git
a/docs/_docs/extensions-and-integrations/change-data-capture-extensions.adoc
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-kafka.adoc
similarity index 50%
rename from
docs/_docs/extensions-and-integrations/change-data-capture-extensions.adoc
rename to
docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-kafka.adoc
index de30adaf959..eb79ddfc2d7 100644
--- a/docs/_docs/extensions-and-integrations/change-data-capture-extensions.adoc
+++
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-kafka.adoc
@@ -12,81 +12,7 @@
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
-= Cross-cluster Replication Extension
-
-== Overview
-link:https://github.com/apache/ignite-extensions/tree/master/modules/cdc-ext[Cross-cluster
Replication Extension] module provides the following ways to set up
cross-cluster replication based on CDC.
-
-.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java[Ignite2IgniteClientCdcStreamer]
- streams changes to destination cluster using
link:thin-clients/java-thin-client[Java Thin Client].
-.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java[Ignite2IgniteCdcStreamer]
- streams changes to destination cluster using client node.
-.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java[Ignite2KafkaCdcStreamer]
combined with
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java[KafkaToIgniteCdcStreamer]
streams changes to destination cluster using
link:https://kafka.apache.org[Apache Kafka] as a transport.
-
-NOTE: Conflict resolver should be defined for each cache replicated between
the clusters.
-
-NOTE: All implementations of the cross-cluster replication support replication
of
link:https://ignite.apache.org/releases/latest/javadoc/org/apache/ignite/binary/BinaryType.html[BinaryTypes]
and
link:https://ignite.apache.org/releases/latest/javadoc/org/apache/ignite/cdc/TypeMapping.html[TypeMappings]
-
-NOTE: To use SQL queries on the destination cluster over CDC-replicated data,
set the same `VALUE_TYPE` in
-link:sql-reference/ddl#create-table[CREATE TABLE] on both source and
destination clusters for each table.
-
-== Ignite to Java Thin Client CDC streamer
-This streamer starts link:thin-clients/java-thin-client[Java Thin Client]
which connects to destination cluster.
-After connection is established, all changes captured by CDC will be
replicated to destination cluster.
-
-NOTE: Instances of `ignite-cdc.sh` with configured streamer should be started
on each server node of source cluster to capture all changes.
-
-image:../../assets/images/integrations/CDC-ignite2igniteClient.svg[]
-
-=== Configuration
-
-[cols="20%,45%,35%",opts="header"]
-|===
-|Name |Description | Default value
-| `caches` | Set of cache names to replicate. | null
-| `destinationClientConfiguration` | Client configuration of thin client that
will connect to destination cluster to replicate changes. | null
-| `onlyPrimary` | Flag to handle changes only on primary node. | `false`
-| `maxBatchSize` | Maximum number of events to be sent to destination cluster
in a single batch. | 1024
-|===
-
-=== Metrics
-
-[cols="25%,75%",opts="header"]
-|===
-|Name |Description
-| `EventsCount` | Count of messages applied to destination cluster.
-| `LastEventTime` | Timestamp of last applied event to destination cluster.
-| `TypesCount` | Count of binary types events applied to destination cluster.
-| `MappingsCount` | Count of mappings events applied to destination cluster
-|===
-
-== Ignite to Ignite CDC streamer
-This streamer starts client node which connects to destination cluster.
-After connection is established, all changes captured by CDC will be
replicated to destination cluster.
-
-NOTE: Instances of `ignite-cdc.sh` with configured streamer should be started
on each server node of source cluster to capture all changes.
-
-image:../../assets/images/integrations/CDC-ignite2ignite.svg[]
-
-=== Configuration
-
-[cols="20%,45%,35%",opts="header"]
-|===
-|Name |Description | Default value
-| `caches` | Set of cache names to replicate. | null
-| `destinationIgniteConfiguration` | Ignite configuration of client nodes that
will connect to destination cluster to replicate changes. | null
-| `onlyPrimary` | Flag to handle changes only on primary node. | `false`
-| `maxBatchSize` | Maximum number of events to be sent to destination cluster
in a single batch. | 1024
-|===
-
-=== Metrics
-
-[cols="25%,75%",opts="header"]
-|===
-|Name |Description
-| `EventsCount` | Count of messages applied to destination cluster.
-| `LastEventTime` | Timestamp of last applied event to destination cluster.
-| `TypesCount` | Count of binary types events applied to destination cluster.
-| `MappingsCount` | Count of mappings events applied to destination cluster
-|===
+= Cross-cluster Replication with Kafka
== CDC replication using Kafka
@@ -175,22 +101,6 @@ It should be just enough to process source cluster load.
Each instance of application will process configured subset of topic
partitions to spread the load.
`KafkaConsumer` for each partition will be created to ensure fair reads.
-==== Installation
-
-. Build `cdc-ext` module with maven:
-+
-```console
- $~/src/ignite-extensions/> mvn clean package -DskipTests
- $~/src/ignite-extensions/> ls modules/cdc-ext/target | grep zip
-ignite-cdc-ext.zip
-```
-
-. Unpack `ignite-cdc-ext.zip` archive to `$IGNITE_HOME` folder.
-
-Now, you have additional binary `$IGNITE_HOME/bin/kafka-to-ignite.sh` and
`$IGNITE_HOME/libs/optional/ignite-cdc-ext` module.
-
-NOTE: Please, enable `ignite-cdc-ext` to be able to run `kafka-to-ignite.sh`.
-
==== Configuration
Application configuration should be done using POJO classes or Spring xml file
like regular Ignite node configuration.
@@ -264,76 +174,3 @@ NOTE:
link:https://kafka.apache.org/documentation/#consumerconfigs_request.timeo
== Fault tolerance
It expected that CDC streamers will be configured with the `onlyPrimary=false`
in most real-world deployments to ensure fault-tolerance.
That means streamer will send the same change several times equal to
`CacheConfiguration#backups` + 1.
-
-== Conflict resolution
-Conflict resolver should be defined for each cache replicated between the
clusters.
-Cross-cluster replication extension has the
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverImpl.java[default]
conflict resolver implementation.
-
-NOTE: Default implementation only select correct entry and never merge.
-
-The default resolver implementation will be used when custom conflict resolver
is not set.
-
-=== Configuration
-
-[cols="20%,45%,35%",opts="header"]
-|===
-|Name |Description | Default value
-| `clusterId` | Local cluster id. Can be any value from 1 to 31. | null
-| `caches` | Set of cache names to handle with this plugin instance. | null
-| `conflictResolveField` | Value field to resolve conflict with. Optional.
Field values must implement `java.lang.Comparable`. | null
-| `conflictResolver` | Custom conflict resolver. Optional. Field must
implement `CacheVersionConflictResolver`. | null
-|===
-
-=== Conflict resolution algorithm
-Replicated changes contain some additional data. Specifically, entry's version
from source cluster is supplied with the changed data.
-Default conflict resolve algorithm based on entry version and
`conflictResolveField`.
-
-==== Conflict resolution based on the entry's version
-This approach provides the eventual consistency guarantee when each entry is
updatable only from a single cluster.
-
-IMPORTANT: This approach does not replicate any updates or removals from the
destination cluster to the source cluster.
-
-.Algorithm:
-.. Changes from the "local" cluster are always win. Any replicated data can be
overridden locally.
-.. If both old and new entry are from the same cluster then entry versions
comparison is used to determine the order.
-.. Conflict resolution failed. Update will be ignored. Failure will be logged.
-
-==== Conflict resolution based on the entry's value field
-This approach provides the eventual consistency guarantee even when entry is
updatable from any cluster.
-
-NOTE: Conflict resolution field, specified by `conflictResolveField`, should
contain a user provided monotonically increasing value such as query id or
timestamp.
-
-IMPORTANT: This approach does not replicate the removals from the destination
cluster to the source cluster, because removes can't be versioned by the field.
-
-.Algorithm:
-.. Changes from the "local" cluster are always win. Any replicated data can be
overridden locally.
-.. If both old and new entry are from the same cluster then entry versions
comparison is used to determine the order.
-.. If `conflictResolveField` is provided then field values comparison is used
to determine the order.
-.. Conflict resolution failed. Update will be ignored. Failure will be logged.
-
-==== Custom conflict resolution rules
-You're able to define your own rules for resolving conflicts based on the
nature of your data and operations.
-This can be particularly useful in more complex situations where the standard
conflict resolution strategies do not apply.
-
-Choosing the right conflict resolution strategy depends on your specific use
case and requires a good understanding of your data and its usage.
-You should consider the nature of your transactions, the rate of change of
your data, and the implications of potential data loss or overwrites when
selecting a conflict resolution strategy.
-
-Custom conflict resolver can be set via `conflictResolver` and allows to
compare or merge the conflict data in any required way.
-
-=== Configuration example
-Configuration is done via Ignite node plugin:
-
-```xml
-<property name="pluginProviders">
- <bean
class="org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverPluginProvider">
- <property name="clusterId" value="1" />
- <property name="caches">
- <util:list>
- <bean class="java.lang.String">
- <constructor-arg type="String" value="queryId" />
- </bean>
- </util:list>
- </property>
- </bean>
-</property>
-```
diff --git
a/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-postgresql.adoc
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-postgresql.adoc
new file mode 100644
index 00000000000..4eb77b1752e
--- /dev/null
+++
b/docs/_docs/extensions-and-integrations/change-data-capture/ignite-to-postgresql.adoc
@@ -0,0 +1,248 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+= Ignite to PostgreSQL replication
+
+== CDC replication to PostgreSql
+
+`IgniteToPostgreSqlCdcConsumer` is a CDC consumer that asynchronously
replicates data from Apache Ignite to PostgreSQL.
+It uses Apache Ignite’s Change Data Capture (CDC) mechanism to track data
changes (`insert`, `update`, `delete`) in specified caches and apply them to
PostgreSQL.
+
+== Key Features
+
+- Per-cache replication (only selected caches are replicated)
+- `onlyPrimary` support (replicates only from primary nodes)
+- Auto table creation in PostgreSQL if needed (`createTables=true`)
+- Batch replication (`batchSize`)
+- User-defined `DataSource` — user configures reliability and transactional
guarantees
+
+== Configuration
+
+Spring XML configuration example (`ignite-to-postgres.xml`):
+
+[source,xml]
+----
+<bean id="cdc.cfg" class="org.apache.ignite.cdc.CdcConfiguration">
+ <property name="consumer">
+ <bean
class="org.apache.ignite.cdc.postgresql.IgniteToPostgreSqlCdcConsumer">
+ <property name="caches">
+ <list>
+ <value>T1</value>
+ <value>T2</value>
+ </list>
+ </property>
+ <property name="batchSize" value="1024" />
+ <property name="onlyPrimary" value="true" />
+ <property name="createTables" value="true" />
+ <property name="dataSource" ref="dataSource" />
+ </bean>
+ </property>
+</bean>
+----
+
+=== Configuration Options
+
+The following settings can be used to configure the behavior of
`IgniteToPostgreSqlCdcConsumer`:
+
+[cols="1,2,1", options="header"]
+|===
+| Setting | Description | Default
+| `dataSource` | JDBC `DataSource` used to connect to the target PostgreSQL
database. Must be provided by the user. | _Required_
+| `caches` | Set of Ignite cache names to replicate. Must be provided by the
user. | _Required_
+| `onlyPrimary` | If `true`, replicates only events originating from the
primary node. Useful to avoid duplicate updates in replicated clusters. | `true`
+| `maxBatchSize` | Maximum number of statements per batch submitted to
PostgreSQL. Affects how many rows are commited in a single `executeBatch()`
call. | `1024`
+| `createTables`| If `true`, missing target tables in PostgreSQL will be
created automatically during startup.| `false`
+|===
+
+We use `PreparedStatement` for batching with `autoCommit` set to `false`,
committing manually after each batch execution.
+
+
+[WARNING]
+====
+Choosing the `dataSource` is the user's responsibility. Consider:
+
+- Required delivery guarantees (e.g., retry logic)
+- High-availability PostgreSQL setup (replicas, failover, etc.)
+====
+
+== Example `dataSource`
+
+[source,xml]
+----
+<bean id="dataSource" class="org.apache.commons.dbcp2.BasicDataSource"
destroy-method="close">
+ <property name="driverClassName" value="org.postgresql.Driver"/>
+ <property name="url"
value="jdbc:postgresql://localhost:5432/ignite_replica"/>
+ <property name="username" value="ignite_user"/>
+ <property name="password" value="secret"/>
+ <property name="initialSize" value="3"/>
+ <property name="maxTotal" value="10"/>
+ <property name="validationQuery" value="SELECT 1"/>
+ <property name="testOnBorrow" value="true"/>
+</bean>
+----
+
+== Schema Conversion
+
+Table schema in PostgreSQL is generated from the `QueryEntity` configured in
Ignite cache.
+Only **one QueryEntity is supported per cache** and is used to generate DDL
and DML operations.
+
+Schema creation occurs once on the first `CdcCacheEvent` if
`createTables=true`.
+
+=== Example: Schema from Ignite to PostgreSQL
+
+[source,java]
+----
+class TestVal {
+ private final String name;
+ private final int val;
+}
+
+QueryEntity qryEntity = new QueryEntity()
+ .setTableName("test_table")
+ .setKeyFieldName("id")
+ .setValueType("demo.TestVal")
+ .addQueryField("id", Integer.class.getName(), null)
+ .addQueryField("name", String.class.getName(), null)
+ .addQueryField("val", Integer.class.getName(), null);
+
+ignite.getOrCreateCache(new CacheConfiguration<Integer, TestVal>("test_table")
+ .setQueryEntities(List.of(qryEntity)));
+----
+
+→ PostgreSQL:
+
+[source,sql]
+----
+CREATE TABLE test_table (
+ id INT PRIMARY KEY,
+ name VARCHAR,
+ val INT
+);
+----
+
+=== Composite Key Example
+
+[source,java]
+----
+class TestKey {
+ private final int id;
+ private final String subId;
+}
+
+class TestVal {
+ private final String name;
+ private final int val;
+}
+
+QueryEntity qryEntity = new QueryEntity()
+ .setTableName("test_table")
+ .setKeyFields(Set.of("id", "subId"))
+ .setValueType("demo.TestVal")
+ .addQueryField("id", Integer.class.getName(), null)
+ .addQueryField("subId", String.class.getName(), null)
+ .addQueryField("name", String.class.getName(), null)
+ .addQueryField("val", Integer.class.getName(), null);
+
+ignite.getOrCreateCache(new CacheConfiguration<TestKey, TestVal>("test_table")
+ .setQueryEntities(List.of(qryEntity)));
+----
+
+→ PostgreSQL:
+
+[source,sql]
+----
+CREATE TABLE test_table (
+ id INT,
+ subId VARCHAR,
+ name VARCHAR,
+ val INT,
+ PRIMARY KEY (id, subId)
+);
+----
+
+== Insert / Update / Delete Events
+
+Insert, update, and delete operations are handled via `CdcEvent`.
+
+=== Upsert with Version Conflict Resolution
+
+Each insert/update is translated into an `INSERT ... ON CONFLICT DO UPDATE`
query, with version-based conflict resolution.
+
+[NOTE]
+====
+A `version` column is automatically added and stored as `BYTEA`.
+
+This version is a 16-byte array based on `CacheEntryVersion` encoded in
big-endian order:
+
+- 4 bytes — `topologyVersion` (int)
+- 8 bytes — `order` (long)
+- 4 bytes — `nodeOrder` (int)
+
+This allows PostgreSQL to compare versions lexicographically:
+
+[source,sql]
+----
+INSERT INTO test_table (id, name, val, version)
+VALUES (1, 'value', 5, E'\x...')
+ON CONFLICT (id) DO UPDATE SET
+ name = EXCLUDED.name,
+ val = EXCLUDED.val
+WHERE test_table.version < EXCLUDED.version;
+----
+====
+
+=== Delete Example
+
+[source,sql]
+----
+DELETE FROM test_table WHERE id = 1;
+----
+
+== Java → PostgreSQL Type Mapping
+
+|===
+| Java Type | PostgreSQL Type | Precision/Scale
+| `java.lang.String` | `VARCHAR(precision)` | Precision only
+| `java.lang.Integer` / `int` | `INT` | None
+| `java.lang.Long` / `long` | `BIGINT` | None
+| `java.lang.Boolean` / `boolean` | `BOOL` | None
+| `java.lang.Double` / `double` | `NUMERIC(precision, scale)` | Precision &
scale
+| `java.lang.Float` / `float` | `NUMERIC(precision, scale)` | Precision & scale
+| `java.math.BigDecimal` | `NUMERIC(precision, scale)` | Precision & scale
+| `java.lang.Short` / `short` | `SMALLINT` | None
+| `java.lang.Byte` / `byte` | `SMALLINT` | None
+| `java.sql.Date` | `DATE` | None
+| `java.sql.Time` | `TIME(precision)` | Precision only
+| `java.sql.Timestamp` | `TIMESTAMP(precision)` | Precision only
+| `java.util.Date` | `TIMESTAMP(precision)` | Precision only
+| `java.util.UUID` | `UUID` | None
+| `java.time.LocalDate` | `DATE` | None
+| `java.time.LocalTime` | `TIME(precision)` | Precision only
+| `java.time.LocalDateTime` | `TIMESTAMP(precision)` | Precision only
+| `java.time.OffsetTime` | `VARCHAR(precision)` | Precision only
+| `java.time.OffsetDateTime` | `TIMESTAMP WITH TIME ZONE` | None
+| `byte[]` | `BYTEA` | None
+|===
+
+[NOTE]
+====
+- Precision and scale values provided in the mapping configuration will be
processed and applied to the generated SQL types where supported.
+- If the Java type is not recognized in the predefined mapping, an exception
will be thrown.
+====
+
+== Limitations
+
+- Only BinaryObject and primitive fields are supported
+- `keepBinary` must be set to `true`
+- Schema evolution is not supported — run with `createTables=true` at startup
diff --git
a/docs/_docs/extensions-and-integrations/change-data-capture/overview.adoc
b/docs/_docs/extensions-and-integrations/change-data-capture/overview.adoc
new file mode 100644
index 00000000000..3ef17e7ff98
--- /dev/null
+++ b/docs/_docs/extensions-and-integrations/change-data-capture/overview.adoc
@@ -0,0 +1,119 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+= Cross-cluster Replication Extension
+
+== Overview
+link:https://github.com/apache/ignite-extensions/tree/master/modules/cdc-ext[Cross-cluster
Replication Extension] module provides the following ways to set up
cross-cluster replication based on CDC.
+
+.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java[Ignite2IgniteClientCdcStreamer]
- streams changes to destination cluster using
link:thin-clients/java-thin-client[Java Thin Client].
+.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java[Ignite2IgniteCdcStreamer]
- streams changes to destination cluster using client node.
+.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java[Ignite2KafkaCdcStreamer]
combined with
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java[KafkaToIgniteCdcStreamer]
streams changes to destination cluster using
link:https://kafka.apache.org[Apache Kafka] as a transport.
+.
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/postgresql/IgniteToPostgreSqlCdcApplier.java[Ignite2PostgreSqlCdcStreamer]
- streams changes to destination PostgreSQL.
+
+NOTE: Conflict resolver should be defined for each cache replicated between
the clusters.
+
+NOTE: All implementations of the cross-cluster replication support replication
of
link:https://ignite.apache.org/releases/latest/javadoc/org/apache/ignite/binary/BinaryType.html[BinaryTypes]
and
link:https://ignite.apache.org/releases/latest/javadoc/org/apache/ignite/cdc/TypeMapping.html[TypeMappings]
+
+NOTE: To use SQL queries on the destination cluster over CDC-replicated data,
set the same `VALUE_TYPE` in
+link:sql-reference/ddl#create-table[CREATE TABLE] on both source and
destination clusters for each table.
+
+== Installation
+
+. Build `cdc-ext` module with maven:
++
+```console
+ $~/src/ignite-extensions/> mvn clean package -DskipTests
+ $~/src/ignite-extensions/> ls modules/cdc-ext/target | grep zip
+ignite-cdc-ext.zip
+```
+
+. Unpack `ignite-cdc-ext.zip` archive to `$IGNITE_HOME` folder.
+
+Now, you have additional binary `$IGNITE_HOME/bin/kafka-to-ignite.sh` and
`$IGNITE_HOME/libs/optional/ignite-cdc-ext` module.
+
+NOTE: Please, enable `ignite-cdc-ext` to be able to run `kafka-to-ignite.sh`.
+
+== Conflict resolution
+Conflict resolver should be defined for each cache replicated between the
clusters.
+Cross-cluster replication extension has the
link:https://github.com/apache/ignite-extensions/blob/master/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverImpl.java[default]
conflict resolver implementation.
+
+NOTE: Default implementation only select correct entry and never merge.
+
+The default resolver implementation will be used when custom conflict resolver
is not set.
+
+=== Configuration
+
+[cols="20%,45%,35%",opts="header"]
+|===
+|Name |Description | Default value
+| `clusterId` | Local cluster id. Can be any value from 1 to 31. | null
+| `caches` | Set of cache names to handle with this plugin instance. | null
+| `conflictResolveField` | Value field to resolve conflict with. Optional.
Field values must implement `java.lang.Comparable`. | null
+| `conflictResolver` | Custom conflict resolver. Optional. Field must
implement `CacheVersionConflictResolver`. | null
+|===
+
+=== Conflict resolution algorithm
+Replicated changes contain some additional data. Specifically, entry's version
from source cluster is supplied with the changed data.
+Default conflict resolve algorithm based on entry version and
`conflictResolveField`.
+
+==== Conflict resolution based on the entry's version
+This approach provides the eventual consistency guarantee when each entry is
updatable only from a single cluster.
+
+IMPORTANT: This approach does not replicate any updates or removals from the
destination cluster to the source cluster.
+
+.Algorithm:
+.. Changes from the "local" cluster are always win. Any replicated data can be
overridden locally.
+.. If both old and new entry are from the same cluster then entry versions
comparison is used to determine the order.
+.. Conflict resolution failed. Update will be ignored. Failure will be logged.
+
+==== Conflict resolution based on the entry's value field
+This approach provides the eventual consistency guarantee even when entry is
updatable from any cluster.
+
+NOTE: Conflict resolution field, specified by `conflictResolveField`, should
contain a user provided monotonically increasing value such as query id or
timestamp.
+
+IMPORTANT: This approach does not replicate the removals from the destination
cluster to the source cluster, because removes can't be versioned by the field.
+
+.Algorithm:
+.. Changes from the "local" cluster are always win. Any replicated data can be
overridden locally.
+.. If both old and new entry are from the same cluster then entry versions
comparison is used to determine the order.
+.. If `conflictResolveField` is provided then field values comparison is used
to determine the order.
+.. Conflict resolution failed. Update will be ignored. Failure will be logged.
+
+==== Custom conflict resolution rules
+You're able to define your own rules for resolving conflicts based on the
nature of your data and operations.
+This can be particularly useful in more complex situations where the standard
conflict resolution strategies do not apply.
+
+Choosing the right conflict resolution strategy depends on your specific use
case and requires a good understanding of your data and its usage.
+You should consider the nature of your transactions, the rate of change of
your data, and the implications of potential data loss or overwrites when
selecting a conflict resolution strategy.
+
+Custom conflict resolver can be set via `conflictResolver` and allows to
compare or merge the conflict data in any required way.
+
+=== Configuration example
+Configuration is done via Ignite node plugin:
+
+```xml
+<property name="pluginProviders">
+ <bean
class="org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverPluginProvider">
+ <property name="clusterId" value="1" />
+ <property name="caches">
+ <util:list>
+ <bean class="java.lang.String">
+ <constructor-arg type="String" value="queryId" />
+ </bean>
+ </util:list>
+ </property>
+ </bean>
+</property>
+```