Skip to content

Commit

Permalink
Added information about exactly-once delivery (#61)
Browse files Browse the repository at this point in the history
* Added information about exactly-once delivery

* Apply suggestions from code review

Co-authored-by: Aditya Mishra <[email protected]>

---------

Co-authored-by: Adalbert Makarovych <[email protected]>
Co-authored-by: Aditya Mishra <[email protected]>
  • Loading branch information
3 people authored Nov 13, 2024
1 parent a3c6131 commit d91cdfb
Show file tree
Hide file tree
Showing 2 changed files with 30 additions and 3 deletions.
24 changes: 24 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -585,3 +585,27 @@ engine variables in SingleStore:
* `snapshots_to_keep` - Defines the number of snapshots to keep for backup and replication.
* `snapshot_trigger_size` - Defines the size of transaction logs in bytes, which, when reached,
triggers a snapshot that is written to disk.
## Achieving Exactly-Once Delivery
**Note:** Exactly-once semantics (EOS) is currently supported only with Kafka Connect in distributed
mode.
With the release of Kafka 3.3.0 and the added support for exactly-once delivery in Kafka Connect,
you can now enable exactly-once delivery for the SingleStore Debezium connector with
minimal configuration. To configure exactly-once delivery for the connector,
perform the following tasks:
1. Set `exactly.once.source.support=enabled` in your Kafka Connect worker configuration. This
ensures EOS is enabled across all workers. If performing a rolling update, first set
`exactly.once.source.support=preparing` on each worker, and then gradually switch them
to `enabled`.
2. Add `exactly.once.support=required` to the connector configuration.
**Note**: If the connector is stopped for an extended period of time, the offset may become stale. In
this scenario, the connector may need to be manually restarted, which triggers a re-execution of
the initial snapshot. This re-snapshotting process can lead to duplicate events, resulting in a loss
of the exactly-once delivery guarantee during the snapshot phase. Users should be aware of this
limitation and closely monitor connector downtime to avoid such scenarios.
Refer to the [Connector Unable to Start](#connector-unable-to-start) section for detailed
information on handling and preventing this issue.
Original file line number Diff line number Diff line change
Expand Up @@ -6,18 +6,16 @@
import io.debezium.connector.common.RelationalBaseSourceConnector;
import io.debezium.relational.RelationalDatabaseConnectorConfig;
import io.debezium.relational.TableId;
import com.singlestore.debezium.SingleStoreConnectorConfig.SnapshotMode;
import java.sql.SQLException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import org.apache.kafka.common.config.Config;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigValue;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.source.ExactlyOnceSupport;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -116,4 +114,9 @@ public List<TableId> getMatchingCollections(Configuration config) {
throw new DebeziumException(e);
}
}

@Override
public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
return ExactlyOnceSupport.SUPPORTED;
}
}

0 comments on commit d91cdfb

Please sign in to comment.