tet123/debezium-embedded/src/main/java/io/debezium/embedded/package-info.java
Randall Hauch 2da5b37f76 DBZ-1 Added support for recording and recovering database schema
Adds a small framework for recording the DDL operations on the schema state (e.g., Tables) as they are read and applied from the log, and when restarting the connector task to recover the accumulated schema state. Where and how the DDL operations are recorded is an abstraction called `DatabaseHistory`, with three options: in-memory (primarily for testing purposes), file-based (for embedded cases and perhaps standalone Kafka Connect uses), and Kafka (for normal Kafka Connect deployments).

The `DatabaseHistory` interface methods take several parameters that are used to construct a `SourceRecord`. The `SourceRecord` type was not used, however, since that would result in this interface (and potential extension mechanism) having a dependency on and exposing the Kafka API. Instead, the more general parameters are used to keep the API simple.

The `FileDatabaseHistory` and `MemoryDatabaseHistory` implementations are both fairly simple, but the `FileDatabaseHistory` relies upon representing each recorded change as a JSON document. This is simple, is easily written to files, allows for recovery of data from the raw file, etc. Although this was done initially using Jackson, the code to read and write the JSON documents required a lot of boilerplate. Instead, the `Document` framework developed during Debezium's very early prototype stages was brought back. It provides a very usable API for working with documents, including the ability to compare documents semantically (e.g., numeric values are converted to be able to compare their numeric values rather than just compare representations) and with or without field order.

The `KafkaDatabaseHistory` is a bit more complicated, since it uses a Kafka broker to record all database schema changes on a single topic with single partition, and then upon restart uses it to recover the history from the dedicated topics. This implementation also records the changes as JSON documents, keeping it simple and independent of the Kafka Connect converters.
2016-02-02 14:27:14 -06:00

61 lines
4.6 KiB
Java

/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.embedded;
/**
* The Debezium Embedded API provides a simple and lightweight way for an application to directly monitor an external database
* and receive all of the data changes that are made to that database.
* <p>
* The primary benefit of embedding a connector is that the application and connector no longer require nor use the external
* systems normally required by Debezium (e.g., Kafka, Kafka Connect, and Zookeeper), so the application architecture becomes
* more straightforward.
* <p>
* However, an embedded connector is not as fault tolerant or scalable, and the connector only monitors the database while
* the application is running. Furthermore, the embedding application is completely responsible for the connector's lifecycle
* and storage of its internal state (e.g., offsets, database schema history, etc.) so that, after the application stops and
* restarts its connector, the connector can continue processing exactly where it left off.
* <h2>Usage</h2>
* <p>
* Applications do not directly work with Debezium connectors, but instead use the {@link io.debezium.embedded.EmbeddedConnector}
* class to configure and build an {@link io.debezium.embedded.EmbeddedConnector} instance that wraps and completely manages
* a standard Debezium connector. The application also provides, among other things, a function that the EmbeddedConnector will
* use to deliver data change events to the application.
* <p>
* Once the application has configured its {@link io.debezium.embedded.EmbeddedConnector} instance and is ready to start receiving
* data change events, the application submits the EmbeddedConnector to an {@link java.util.concurrent.Executor} or
* {@link java.util.concurrent.ExecutorService} managed by the application. The EmbeddedConnector's
* {@link io.debezium.embedded.EmbeddedConnector#run()} method will start the standard Debezium connector and continuously
* deliver any data changes events to the application.
* <p>
* When the application is ready to shut down the connector, it should call {@link EmbeddedConnector#stop()} on the
* EmbeddedConnector, which will then stop monitoring the source database, complete any current work, and gracefully shut down.
* The application can wait for the connector to complete by using the
* {@link io.debezium.embedded.EmbeddedConnector#await(long, java.util.concurrent.TimeUnit)} method.
* <h2>Storing connector state</h2>
* <p>
* As Debezium connectors operate, they keep track of which information from the source database they have processed, and they
* record this <em>offset information</em> in an {@link org.apache.kafka.connect.storage.OffsetBackingStore}. Kafka Connect
* provides several implementations that can be used by an application, including a
* {@link org.apache.kafka.connect.storage.FileOffsetBackingStore file-based store} and an
* {@link org.apache.kafka.connect.storage.MemoryOffsetBackingStore memory-based store}. For most applications the memory-based
* store will not be sufficient, since when the application shuts down all offset information will be lost. Instead, most
* applications should use the file-based store (or another persistent implementation of
* {@link org.apache.kafka.connect.storage.OffsetBackingStore}) so that all offset information is persisted after the application
* terminates and can be read upon restart.
* <p>
* Some Debezium connectors to relational databases may also keep track of all changes to the database's schema so that it has
* the correct table structure for any point in time as it reads the transaction logs. This is critical information, since the
* data being read from the transaction log reflects the database structure at the time those records were written in the log,
* and the database's table structure may have changed since that point in time. These connectors use a
* {@link io.debezium.relational.history.DatabaseHistory} store to persist the database schema changes and the offsets at which
* the changes are recorded. This way, no matter at which offset the database connector starts reading the transaction log, the
* connector will have the correct database schema for that point in time. And, just like with the
* {@link org.apache.kafka.connect.storage.OffsetBackingStore}, the application must provide the EmbeddedConnector with a
* {@link io.debezium.relational.history.DatabaseHistory} implementation such as the
* {@link io.debezium.relational.history.FileDatabaseHistory} that stores the schema changes on the local file system.
*/