├── .gitignore ├── README.md ├── aggregation ├── pom.xml └── src │ └── main │ ├── java │ └── io │ │ └── debezium │ │ └── examples │ │ └── aggregation │ │ ├── StreamingDbAggregator.java │ │ └── db │ │ └── DBCPDataSource.java │ └── resources │ └── avro │ ├── aggregation │ └── orders │ │ └── Order.avsc │ └── dbserver1 │ └── inventory │ └── orders │ ├── Key.avsc │ └── Value.avsc ├── connect-jdbc-es ├── Dockerfile └── confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz ├── docker-compose-postgres.yaml ├── es-sink-aggregated-orders.json ├── es-sink-dbserver1.inventory.customers.json ├── img └── overview.png ├── register-postgres-smt.json └── register-postgres.json /.gitignore: -------------------------------------------------------------------------------- 1 | *.iml 2 | .idea 3 | target 4 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Sink information from Postgres towards ElasticSearch via Debezium 2 | 3 | ## Introduction 4 | 5 | This is a proof of concept project used for showcasing how to eventually sync data from Postgres database towards ElasticSearch via [Debezium](https://debezium.io/). 6 | 7 | The main idea is to try to sync not only one database entity, but an aggregation of multiple entities towards 8 | a downstream service/database. 9 | 10 | The [Debezium](https://debezium.io/) Kafka connector can read from the transaction log of the database and provide 11 | for each entity a corresponding topic where the state of the 12 | entity before and after the transaction is being logged. 13 | Nevertheless, within downstream services (e.g. : a search index) there is often 14 | needed an aggregation of multiple related database entities. 15 | 16 | This project acts as a proof of concept for such an aggregation by synchronizing the orders performed in an _inventory_ database into an [Elasticsearch](https://www.elastic.co/) search index. 17 | 18 | When performing CRUD operations on the orders table from the Postgres database, the changed orders (along with their related entities) will be eventually synchronized (in near-real-time) towards the search index. 19 | 20 | 21 | The database is comprised of the following tables: 22 | 23 | - orders 24 | - customers 25 | - products 26 | 27 | where each order has a foreign key towards a customer and to a product. 28 | 29 | 30 | 31 | 32 | 33 | The topic previously mentioned has been already researched at Debezium: 34 | 35 | [Domain Data Driven aggregates with Debezium](https://debezium.io/blog/2018/03/08/creating-ddd-aggregates-with-debezium-and-kafka-streams/) 36 | 37 | >There are use cases however, where things are a bit more tricky. It is sometimes useful to share information across services and data stores by means of so-called aggregates, which are a concept/pattern defined by domain-driven design (DDD). In general, a DDD aggregate is used to transfer state which can be comprised of multiple different domain objects that are together treated as a single unit of information. 38 | > 39 | >Concrete examples are: 40 | >- customers and their addresses which are represented as a customer record aggregate storing a customer and a list of addresses 41 | >- orders and corresponding line items which are represented as an order record aggregate storing an order and all its line items 42 | 43 | https://github.com/debezium/debezium-examples/tree/master/kstreams 44 | 45 | 46 | This project follows a rather different approach, by retrieving the entities related to the orders via *JDBC* select statements. 47 | At the cost of reading explicitly (in the aggregation Kafka Stream application) the associated entities via JDBC, there can be achieved the need of having (eventually) up-to-date aggregated order entries in the search index. 48 | 49 | 50 | 51 | [Avro](https://avro.apache.org/) is being used for data serialization over Apache Kafka. 52 | 53 | ## Overview 54 | 55 | ![overview](./img/overview.png) 56 | 57 | 58 | ## Installation 59 | 60 | 61 | ### Create own version of debezium/connect-jdbc-es in order to have the elasticsearch configuration for kafka-connect 62 | 63 | ``` 64 | export DEBEZIUM_VERSION=0.9 65 | cd connect-jdbc-es 66 | docker build -t debezium/connect-jdbc-es:0.9 . 67 | ``` 68 | 69 | This image is then referenced in the docker-compose-postgres.yaml file. 70 | 71 | 72 | ### Start the cluster as defined in http://debezium.io/docs/tutorial/ 73 | ``` 74 | export DEBEZIUM_VERSION=0.9 75 | docker-compose -f docker-compose-postgres.yaml up 76 | ``` 77 | 78 | 79 | ### Start Postgres connector 80 | ``` 81 | curl -i -X POST -H "Accept:application/json" -H "Content-Type:application/json" http://localhost:8083/connectors/ -d @register-postgres-smt.json 82 | ``` 83 | 84 | 85 | ### Start Elasticsearch connector 86 | 87 | ``` 88 | curl -i -X POST -H "Accept:application/json" -H "Content-Type:application/json" http://localhost:8083/connectors/ -d @es-sink-aggregated-orders.json 89 | ``` 90 | 91 | 92 | ### Build the aggregation kafka streaming application 93 | 94 | ``` 95 | cd aggregation 96 | mvn clean install 97 | ``` 98 | 99 | ### Register the aggregated order subject in the schema registry 100 | 101 | ``` 102 | cd aggregation 103 | mvn schema-registry:register 104 | ``` 105 | 106 | ### Run the kafka stream application 107 | 108 | ```command line parameters 109 | dbserver1.inventory.orders aggregated-orders localhost:9092 110 | ``` 111 | 112 | 113 | ### Verify that the data has been synced to ElasticSearch 114 | 115 | ``` 116 | curl http://localhost:9200/aggregated-orders/_search | jq 117 | ``` 118 | 119 | 120 | 121 | ### Shut down the cluster 122 | ``` 123 | docker-compose -f docker-compose-postgres.yaml down 124 | ``` 125 | 126 | 127 | 128 | ## Useful commands 129 | 130 | Consume messages from a Debezium topic 131 | ``` 132 | docker-compose -f docker-compose-postgres.yaml exec kafka /kafka/bin/kafka-console-consumer.sh \ 133 | --bootstrap-server kafka:9092 \ 134 | --from-beginning \ 135 | --property print.key=true \ 136 | --topic dbserver1.inventory.customers 137 | ``` 138 | 139 | 140 | Consume messages from a Debezium topic with Avro 141 | 142 | ``` 143 | docker-compose -f docker-compose-postgres.yaml exec schema-registry /usr/bin/kafka-avro-console-consumer \ 144 | --bootstrap-server kafka:9092 \ 145 | --from-beginning \ 146 | --property print.key=true \ 147 | --property schema.registry.url=http://schema-registry:8081 \ 148 | --topic dbserver1.inventory.orders 149 | ``` 150 | 151 | 152 | Modify records in the database via Postgres client 153 | 154 | ``` 155 | docker-compose -f docker-compose-postgres.yaml exec postgres env PGOPTIONS="--search_path=inventory" bash -c 'psql -U $POSTGRES_USER postgres' 156 | ``` 157 | 158 | 159 | 160 | 161 | 162 | See the kafka connect connectors configuration 163 | 164 | ``` 165 | curl -H "Accept:application/json" localhost:8083/connectors/ | jq 166 | ``` 167 | 168 | See the elasticsearch kafka-connect connector configuration 169 | 170 | ``` 171 | curl -H "Accept:application/json" localhost:8083/connectors/elastic-sink | jq 172 | ``` 173 | 174 | See the logs of the kafka-connect container 175 | 176 | ``` 177 | docker-compose -f docker-compose-postgres.yaml logs connect 178 | ``` 179 | 180 | See the contents of the `dbserver1.inventory.customers` elasticsearch index 181 | 182 | ``` 183 | curl http://localhost:9200/dbserver1.inventory.customers/_search | jq 184 | ``` 185 | 186 | 187 | See the aggregated orders where `Sally` terms comes along 188 | 189 | ``` 190 | curl http://localhost:9200/aggregated-orders/_search\?q\=Sally | jq 191 | ``` 192 | 193 | 194 | ``` 195 | curl http://localhost:9200/aggregated-orders/_search\?q\=customer.first_name:Sally | jq 196 | 197 | ``` 198 | 199 | Reset offsets of the kafka stream application 200 | 201 | More details here: https://www.confluent.io/blog/data-reprocessing-with-kafka-streams-resetting-a-streams-application/ 202 | 203 | ``` 204 | ./kafka-streams-application-reset --application-id streaming-db-aggregator \ 205 | --input-topics dbserver1.inventory.orders \ 206 | --intermediate-topics streaming-db-aggregator-dbserver1.inventory.ordersSTATE-STORE-0000000000-changelog \ 207 | --bootstrap-servers localhost:9092 208 | ``` 209 | 210 | Show topics 211 | 212 | ``` 213 | ./kafka-topics --list --zookeeper localhost:2181 214 | ``` 215 | 216 | 217 | See the subjects in the schema registry 218 | 219 | 220 | ``` 221 | curl -X GET http://localhost:8081/subjects/ 222 | ``` 223 | 224 | 225 | See a subject detail in the schema registry 226 | 227 | ``` 228 | curl -X GET http://localhost:8081/subjects/order/versions/1 | jq '.schema | fromjson' 229 | ``` 230 | 231 | 232 | Running the aggregation stream 233 | 234 | ```command line parameters 235 | dbserver1.inventory.orders aggregated-orders localhost:9092 236 | ``` 237 | 238 | 239 | ## Tutorials 240 | 241 | Schema Registry tutorial 242 | https://docs.confluent.io/current/schema-registry/schema_registry_tutorial.html 243 | 244 | 245 | 246 | Debezium Tutorial 247 | 248 | https://github.com/debezium/debezium-examples/blob/master/tutorial/README.md 249 | 250 | 251 | 252 | Streaming from Debezium to ElasticSearch 253 | 254 | https://debezium.io/blog/2018/01/17/streaming-to-elasticsearch/ 255 | 256 | 257 | ## Further resources 258 | 259 | https://debezium.io/blog/2019/02/19/reliable-microservices-data-exchange-with-the-outbox-pattern/ 260 | 261 | 262 | The Outbox Pattern 263 | The idea of this approach is to have an "outbox" table in the service’s database. When receiving a request for placing a purchase order, not only an INSERT into the PurchaseOrder table is done, but, as part of the same transaction, also a record representing the event to be sent is inserted into that outbox table. 264 | 265 | The record describes an event that happened in the service, for instance it could be a JSON structure representing the fact that a new purchase order has been placed, comprising data on the order itself, its order lines as well as contextual information such as a use case identifier. By explicitly emitting events via records in the outbox table, it can be ensured that events are structured in a way suitable for external consumers. This also helps to make sure that event consumers won’t break when for instance altering the internal domain model or the PurchaseOrder table. 266 | 267 | https://github.com/debezium/debezium-examples/tree/master/outbox 268 | 269 | 270 | 271 | 272 | 273 | https://dunningpostor.com/streaming-postgres-to-kafka-and-maintaining-consistency-and-transaction-isolation/ 274 | 275 | For instance, Debezium (and bottledwater) creates separate Kafka topics for each Postgres table. So when a transaction is committed that affects 5 tables, 5 different corresponding Kafka topics will each get at least 1 event appended to them (depending on how many records in that table were updated/inserted/removed). 276 | 277 | Any stream processor consuming these Postgres-driven Kafka topics that wants to maintain the same data consistency and transaction isolation of the source database will need to be very careful not to produce data from only part of the original database transaction. 278 | 279 | 280 | 281 | https://stackoverflow.com/questions/56250934/stream-join-example-with-apache-kafka 282 | 283 | customer & answer join over KSQL -------------------------------------------------------------------------------- /aggregation/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4 | 4.0.0 5 | 6 | io.debezium.examples.pg2es 7 | aggregation 8 | 0.1-SNAPSHOT 9 | 10 | 11 | 12 | 1.8.2 13 | ${project.basedir}/src/main/resources/avro/ 14 | 15 | 5.2.1 16 | 17 | 3.5.1 18 | 3.0.2 19 | 1.0.0 20 | UTF-8 21 | UTF-8 22 | 1.8 23 | 24 | 25 | 26 | 27 | 28 | org.apache.kafka 29 | kafka_2.11 30 | ${apache.kafka.version} 31 | 32 | 33 | 34 | org.apache.kafka 35 | kafka-streams 36 | ${apache.kafka.version} 37 | 38 | 39 | 40 | org.apache.kafka 41 | kafka-clients 42 | ${apache.kafka.version} 43 | 44 | 45 | 46 | 47 | 48 | org.apache.commons 49 | commons-dbcp2 50 | 2.6.0 51 | 52 | 53 | 54 | 55 | 56 | org.postgresql 57 | postgresql 58 | 42.2.5 59 | 60 | 61 | 62 | 63 | org.apache.avro 64 | avro 65 | ${avro.version} 66 | 67 | 68 | 69 | 70 | io.confluent 71 | kafka-avro-serializer 72 | ${confluent.version} 73 | 74 | 75 | 76 | io.confluent 77 | kafka-streams-avro-serde 78 | ${confluent.version} 79 | 80 | 81 | 82 | io.confluent 83 | kafka-streams-avro-serde 84 | ${confluent.version} 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | org.apache.maven.plugins 93 | maven-compiler-plugin 94 | ${mvn.compiler.version} 95 | 96 | ${java.version} 97 | ${java.version} 98 | 99 | 100 | 101 | org.apache.avro 102 | avro-maven-plugin 103 | ${avro.version} 104 | 105 | 106 | generate-sources 107 | 108 | schema 109 | 110 | 111 | PRIVATE 112 | ${avro.schema.dir} 113 | ${project.build.directory}/generated-sources 114 | String 115 | 116 | 117 | 118 | 119 | 120 | 125 | 126 | io.confluent 127 | kafka-schema-registry-maven-plugin 128 | 5.2.1 129 | 130 | 131 | http://localhost:8081 132 | 133 | src/main/resources/avro 134 | 135 | src/main/resources/avro/aggregation/orders/Order.avsc 136 | 137 | 138 | 139 | register 140 | 141 | 142 | 143 | 144 | 145 | 146 | io.confluent 147 | kafka-schema-registry-maven-plugin 148 | ${confluent.version} 149 | 150 | 151 | http://localhost:8081 152 | 153 | 154 | ${project.basedir}/src/main/resources/avro/dbserver1/inventory/orders/Key.avsc 155 | ${project.basedir}/src/main/resources/avro/dbserver1/inventory/orders/Value.avsc 156 | 157 | 158 | 159 | test-compatibility 160 | 161 | 162 | 163 | 164 | 165 | 166 | org.apache.maven.plugins 167 | maven-dependency-plugin 168 | ${mvn.dependency.version} 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | confluent 177 | Confluent 178 | https://packages.confluent.io/maven/ 179 | 180 | 181 | 182 | 183 | confluent 184 | https://packages.confluent.io/maven/ 185 | 186 | 187 | 188 | 189 | -------------------------------------------------------------------------------- /aggregation/src/main/java/io/debezium/examples/aggregation/StreamingDbAggregator.java: -------------------------------------------------------------------------------- 1 | package io.debezium.examples.aggregation; 2 | 3 | import aggregation.orders.Customer; 4 | import aggregation.orders.Order; 5 | import aggregation.orders.Product; 6 | import dbserver1.inventory.orders.Key; 7 | import dbserver1.inventory.orders.Value; 8 | import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig; 9 | import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde; 10 | import io.debezium.examples.aggregation.db.DBCPDataSource; 11 | import org.apache.kafka.clients.CommonClientConfigs; 12 | import org.apache.kafka.clients.consumer.ConsumerConfig; 13 | import org.apache.kafka.common.serialization.Serde; 14 | import org.apache.kafka.streams.KafkaStreams; 15 | import org.apache.kafka.streams.StreamsBuilder; 16 | import org.apache.kafka.streams.StreamsConfig; 17 | import org.apache.kafka.streams.kstream.KTable; 18 | import org.apache.kafka.streams.kstream.Printed; 19 | import org.apache.kafka.streams.kstream.Produced; 20 | 21 | import java.sql.Connection; 22 | import java.sql.PreparedStatement; 23 | import java.sql.ResultSet; 24 | import java.sql.SQLException; 25 | import java.util.Collections; 26 | import java.util.Map; 27 | import java.util.Properties; 28 | 29 | public class StreamingDbAggregator { 30 | 31 | public static void main(String[] args) { 32 | if (args.length != 3) { 33 | System.err.println("usage: java -jar " 34 | + StreamingDbAggregator.class.getName() 35 | + " "); 36 | System.exit(-1); 37 | } 38 | 39 | final String parentTopic = args[0]; 40 | final String aggregationTopic = args[1]; 41 | final String bootstrapServers = args[2]; 42 | 43 | Properties props = new Properties(); 44 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streaming-db-aggregator"); 45 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); 46 | props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024); 47 | props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); 48 | props.put(CommonClientConfigs.METADATA_MAX_AGE_CONFIG, 500); 49 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 50 | 51 | 52 | // Where to find the Confluent schema registry instance(s) 53 | props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "http://localhost:8081"); 54 | // Specify default (de)serializers for record keys and for record values. 55 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); 56 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); 57 | 58 | StreamsBuilder builder = new StreamsBuilder(); 59 | 60 | //1) read parent topic i.e. orders as ktable 61 | KTable ordersTable = builder.table(parentTopic); 62 | 63 | ordersTable.toStream().print(Printed.toSysOut()); 64 | 65 | //2) map the read organisation entity to the aggregated order dto 66 | KTable aggregatedOrdersTable = ordersTable 67 | .mapValues(StreamingDbAggregator::createOrderDto); 68 | 69 | 70 | final Map serdeConfig = Collections.singletonMap("schema.registry.url", 71 | "http://localhost:8081"); 72 | 73 | final Serde keyAvroSerde = new SpecificAvroSerde<>(); 74 | keyAvroSerde.configure(serdeConfig, true); // `true` for record keys 75 | final Serde aggregatedOrderAvroSerde = new SpecificAvroSerde<>(); 76 | aggregatedOrderAvroSerde.configure(serdeConfig, false); // `false` for record values 77 | 78 | 79 | aggregatedOrdersTable.toStream().to(aggregationTopic, 80 | Produced.with(keyAvroSerde, aggregatedOrderAvroSerde)); 81 | aggregatedOrdersTable.toStream().print(Printed.toSysOut()); 82 | 83 | final KafkaStreams streams = new KafkaStreams(builder.build(), props); 84 | 85 | // Delete the application's local state. 86 | // Note: In real application you'd call `cleanUp()` only under certain conditions. 87 | // See Confluent Docs for more details: 88 | // https://docs.confluent.io/current/streams/developer-guide/app-reset-tool.html#step-2-reset-the-local-environments-of-your-application-instances 89 | //streams.cleanUp(); 90 | 91 | streams.start(); 92 | 93 | Runtime.getRuntime().addShutdownHook(new Thread(streams::close)); 94 | } 95 | 96 | 97 | private static Order createOrderDto(Value orderTuple) { 98 | Product productDto = getProduct(orderTuple.getProductId()); 99 | Customer customerDto = getCustomer(orderTuple.getPurchaser()); 100 | return new Order(orderTuple.getId(), 101 | orderTuple.getOrderDate(), 102 | customerDto, 103 | orderTuple.getQuantity(), 104 | productDto); 105 | } 106 | 107 | 108 | private static Product getProduct(Integer productId) { 109 | if (productId == null) { 110 | return null; 111 | } 112 | String sql = "SELECT id, name, description, weight from products where id = ?"; 113 | try (Connection connection = DBCPDataSource.getConnection(); 114 | PreparedStatement pstmt = connection.prepareStatement(sql);) { 115 | pstmt.setInt(1, productId); 116 | try (ResultSet resultSet = pstmt.executeQuery();) { 117 | if (resultSet.next()) { 118 | return new Product( 119 | resultSet.getInt(1), 120 | resultSet.getString(2), 121 | resultSet.getString(3), 122 | resultSet.getFloat(4) 123 | ); 124 | } 125 | } 126 | } 127 | catch (SQLException e) { 128 | throw new RuntimeException("SQL Exception occurred", e); 129 | } 130 | 131 | return null; 132 | } 133 | 134 | 135 | private static Customer getCustomer(Integer customerId) { 136 | if (customerId == null) { 137 | return null; 138 | } 139 | String sql = "SELECT id, first_name, last_name, email from customers where id = ?"; 140 | try (Connection connection = DBCPDataSource.getConnection(); 141 | PreparedStatement pstmt = connection.prepareStatement(sql);) { 142 | pstmt.setInt(1, customerId); 143 | try (ResultSet resultSet = pstmt.executeQuery();) { 144 | if (resultSet.next()) { 145 | return new Customer( 146 | resultSet.getInt(1), 147 | resultSet.getString(2), 148 | resultSet.getString(3), 149 | resultSet.getString(4) 150 | ); 151 | } 152 | } 153 | } 154 | catch (SQLException e) { 155 | throw new RuntimeException("SQL Exception occurred", e); 156 | } 157 | 158 | return null; 159 | } 160 | } -------------------------------------------------------------------------------- /aggregation/src/main/java/io/debezium/examples/aggregation/db/DBCPDataSource.java: -------------------------------------------------------------------------------- 1 | package io.debezium.examples.aggregation.db; 2 | 3 | import org.apache.commons.dbcp2.BasicDataSource; 4 | 5 | import java.sql.Connection; 6 | import java.sql.SQLException; 7 | 8 | 9 | public class DBCPDataSource { 10 | 11 | private static BasicDataSource ds = new BasicDataSource(); 12 | 13 | static { 14 | ds.setUrl("jdbc:postgresql://localhost/postgres?currentSchema=inventory"); 15 | ds.setUsername("postgres"); 16 | ds.setPassword("postgres"); 17 | ds.setMinIdle(5); 18 | ds.setMaxIdle(10); 19 | ds.setMaxOpenPreparedStatements(100); 20 | } 21 | 22 | public static Connection getConnection() throws SQLException { 23 | return ds.getConnection(); 24 | } 25 | 26 | private DBCPDataSource(){ } 27 | } -------------------------------------------------------------------------------- /aggregation/src/main/resources/avro/aggregation/orders/Order.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type": "record", 3 | "name": "Order", 4 | "namespace": "aggregation.orders", 5 | "fields": [ 6 | { 7 | "name": "id", 8 | "type": "int" 9 | }, 10 | { 11 | "name": "order_date", 12 | "type": { 13 | "type": "int", 14 | "connect.version": 1, 15 | "connect.name": "io.debezium.time.Date" 16 | } 17 | }, 18 | { 19 | "name": "customer", 20 | "type":[ 21 | "null", 22 | { 23 | "namespace": "aggregation.orders", 24 | "type": "record", 25 | "name": "Customer", 26 | "fields": [ 27 | {"name": "id", "type": "int"}, 28 | {"name": "first_name", "type": "string"}, 29 | {"name": "last_name", "type": "string"}, 30 | {"name": "email", "type": "string"} 31 | ] 32 | } 33 | ], 34 | "default": null 35 | }, 36 | { 37 | "name": "quantity", 38 | "type": "int" 39 | }, 40 | { 41 | "name": "product", 42 | "type": [ 43 | "null", 44 | { 45 | "namespace": "aggregation.orders", 46 | "name": "Product", 47 | "type": "record", 48 | "fields": [ 49 | {"name": "id", "type": "int"}, 50 | {"name": "name", "type": "string"}, 51 | {"name": "description", "type": "string"}, 52 | {"name": "weight", "type": "float"} 53 | ] 54 | } 55 | ], 56 | "default": null 57 | } 58 | ] 59 | } -------------------------------------------------------------------------------- /aggregation/src/main/resources/avro/dbserver1/inventory/orders/Key.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type": "record", 3 | "name": "Key", 4 | "namespace": "dbserver1.inventory.orders", 5 | "fields": [ 6 | { 7 | "name": "id", 8 | "type": "int" 9 | } 10 | ], 11 | "connect.name": "dbserver1.inventory.orders.Key" 12 | } -------------------------------------------------------------------------------- /aggregation/src/main/resources/avro/dbserver1/inventory/orders/Value.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type": "record", 3 | "name": "Value", 4 | "namespace": "dbserver1.inventory.orders", 5 | "fields": [ 6 | { 7 | "name": "id", 8 | "type": "int" 9 | }, 10 | { 11 | "name": "order_date", 12 | "type": { 13 | "type": "int", 14 | "connect.version": 1, 15 | "connect.name": "io.debezium.time.Date" 16 | } 17 | }, 18 | { 19 | "name": "purchaser", 20 | "type": "int" 21 | }, 22 | { 23 | "name": "quantity", 24 | "type": "int" 25 | }, 26 | { 27 | "name": "product_id", 28 | "type": "int" 29 | } 30 | ], 31 | "connect.name": "dbserver1.inventory.orders.Value" 32 | } -------------------------------------------------------------------------------- /connect-jdbc-es/Dockerfile: -------------------------------------------------------------------------------- 1 | FROM debezium/connect-base:0.9 2 | 3 | LABEL maintainer="Debezium Community" 4 | 5 | ENV DEBEZIUM_VERSION="0.9.5.Final" \ 6 | CONFLUENT_VERSION="5.2.1" \ 7 | MAVEN_REPO_CORE="https://repo1.maven.org/maven2" \ 8 | MAVEN_DEP_DESTINATION=$KAFKA_CONNECT_PLUGINS_DIR \ 9 | MONGODB_MD5=a9bde9c4173d4c4f3c0adc9d97a795ee \ 10 | MYSQL_MD5=720b1396358fbdc59bce953f47d3c53f \ 11 | POSTGRES_MD5=a838ae664b7d8d050419d3e638e64273 \ 12 | SQLSERVER_MD5=f21ef11e7b3e34736d428eb1d6ce5450 \ 13 | ORACLE_MD5=1b93c90502d5d369e5a7173759de2504 14 | 15 | COPY confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz 16 | 17 | RUN docker-maven-download debezium mongodb "$DEBEZIUM_VERSION" "$MONGODB_MD5" && \ 18 | docker-maven-download debezium mysql "$DEBEZIUM_VERSION" "$MYSQL_MD5" && \ 19 | docker-maven-download debezium postgres "$DEBEZIUM_VERSION" "$POSTGRES_MD5" && \ 20 | docker-maven-download debezium sqlserver "$DEBEZIUM_VERSION" "$SQLSERVER_MD5" && \ 21 | docker-maven-download debezium oracle "$DEBEZIUM_VERSION" "$ORACLE_MD5" && \ 22 | tar -xzf confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz -C "$MAVEN_DEP_DESTINATION" 23 | 24 | -------------------------------------------------------------------------------- /connect-jdbc-es/confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/mariusneo/postgres-to-elastic-via-debezium/816d7ec3be0ff760942ff11aa345edeb3b9bf55b/connect-jdbc-es/confluentinc-kafka-connect-elasticsearch-5.2.1.tar.gz -------------------------------------------------------------------------------- /docker-compose-postgres.yaml: -------------------------------------------------------------------------------- 1 | version: '2' 2 | services: 3 | zookeeper: 4 | image: debezium/zookeeper:${DEBEZIUM_VERSION} 5 | ports: 6 | - 2181:2181 7 | - 2888:2888 8 | - 3888:3888 9 | kafka: 10 | image: debezium/kafka:${DEBEZIUM_VERSION} 11 | ports: 12 | - 9092:9092 13 | links: 14 | - zookeeper 15 | environment: 16 | - ZOOKEEPER_CONNECT=zookeeper:2181 17 | postgres: 18 | image: debezium/example-postgres:${DEBEZIUM_VERSION} 19 | ports: 20 | - 5432:5432 21 | environment: 22 | - POSTGRES_USER=postgres 23 | - POSTGRES_PASSWORD=postgres 24 | schema-registry: 25 | image: confluentinc/cp-schema-registry:5.2.1 26 | ports: 27 | - 8181:8181 28 | - 8081:8081 29 | environment: 30 | - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:2181 31 | - SCHEMA_REGISTRY_HOST_NAME=schema-registry 32 | - SCHEMA_REGISTRY_LISTENERS=http://schema-registry:8081 33 | links: 34 | - zookeeper 35 | connect: 36 | image: debezium/connect-jdbc-es:${DEBEZIUM_VERSION} 37 | ports: 38 | - 8083:8083 39 | links: 40 | - kafka 41 | - postgres 42 | environment: 43 | - BOOTSTRAP_SERVERS=kafka:9092 44 | - GROUP_ID=1 45 | - CONFIG_STORAGE_TOPIC=my_connect_configs 46 | - OFFSET_STORAGE_TOPIC=my_connect_offsets 47 | - STATUS_STORAGE_TOPIC=my_connect_statuses 48 | - KEY_CONVERTER=io.confluent.connect.avro.AvroConverter 49 | - VALUE_CONVERTER=io.confluent.connect.avro.AvroConverter 50 | - INTERNAL_KEY_CONVERTER=org.apache.kafka.connect.json.JsonConverter 51 | - INTERNAL_VALUE_CONVERTER=org.apache.kafka.connect.json.JsonConverter 52 | - CONNECT_KEY_CONVERTER_SCHEMA_REGISTRY_URL=http://schema-registry:8081 53 | - CONNECT_VALUE_CONVERTER_SCHEMA_REGISTRY_URL=http://schema-registry:8081 54 | elasticsearch: 55 | image: docker.elastic.co/elasticsearch/elasticsearch:6.0.1 56 | environment: 57 | - cluster.name=docker-cluster 58 | - bootstrap.memory_lock=true 59 | - discovery.type=single-node 60 | - "ES_JAVA_OPTS=-Xms512m -Xmx512m" 61 | ulimits: 62 | memlock: 63 | soft: -1 64 | hard: -1 65 | ports: 66 | - "9200:9200" 67 | - "9300:9300" 68 | 69 | -------------------------------------------------------------------------------- /es-sink-aggregated-orders.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "elastic-sink", 3 | "config": { 4 | "connector.class": "io.confluent.connect.elasticsearch.ElasticsearchSinkConnector", 5 | "tasks.max": "1", 6 | "topics": "aggregated-orders", 7 | "connection.url": "http://elasticsearch:9200", 8 | "transforms": "unwrap,key", 9 | "transforms.unwrap.type": "io.debezium.transforms.UnwrapFromEnvelope", 10 | "transforms.key.type": "org.apache.kafka.connect.transforms.ExtractField$Key", 11 | "transforms.key.field": "id", 12 | "key.ignore": "false", 13 | "type.name": "order" 14 | } 15 | } 16 | 17 | -------------------------------------------------------------------------------- /es-sink-dbserver1.inventory.customers.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "elastic-sink", 3 | "config": { 4 | "connector.class": "io.confluent.connect.elasticsearch.ElasticsearchSinkConnector", 5 | "tasks.max": "1", 6 | "topics": "dbserver1.inventory.customers", 7 | "connection.url": "http://elasticsearch:9200", 8 | "transforms": "unwrap,key", 9 | "transforms.unwrap.type": "io.debezium.transforms.UnwrapFromEnvelope", 10 | "transforms.key.type": "org.apache.kafka.connect.transforms.ExtractField$Key", 11 | "transforms.key.field": "id", 12 | "key.ignore": "false", 13 | "type.name": "customer" 14 | } 15 | } 16 | 17 | -------------------------------------------------------------------------------- /img/overview.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/mariusneo/postgres-to-elastic-via-debezium/816d7ec3be0ff760942ff11aa345edeb3b9bf55b/img/overview.png -------------------------------------------------------------------------------- /register-postgres-smt.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "inventory-connector", 3 | "config": { 4 | "connector.class": "io.debezium.connector.postgresql.PostgresConnector", 5 | "tasks.max": "1", 6 | "database.hostname": "postgres", 7 | "database.port": "5432", 8 | "database.user": "postgres", 9 | "database.password": "postgres", 10 | "database.dbname" : "postgres", 11 | "database.server.name": "dbserver1", 12 | "schema.whitelist": "inventory", 13 | "table.whitelist": "inventory.orders", 14 | "transforms": "unwrap", 15 | "transforms.unwrap.type":"io.debezium.transforms.UnwrapFromEnvelope", 16 | "transforms.unwrap.drop.tombstones":"false" 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /register-postgres.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "inventory-connector", 3 | "config": { 4 | "connector.class": "io.debezium.connector.postgresql.PostgresConnector", 5 | "tasks.max": "1", 6 | "database.hostname": "postgres", 7 | "database.port": "5432", 8 | "database.user": "postgres", 9 | "database.password": "postgres", 10 | "database.dbname" : "postgres", 11 | "database.server.name": "dbserver1", 12 | "schema.whitelist": "inventory" 13 | } 14 | } 15 | --------------------------------------------------------------------------------