Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
## I/Os

* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
* DebeziumIO (Java): added `OffsetRetainer` interface and `FileSystemOffsetRetainer` implementation to persist and restore CDC offsets across pipeline restarts, and exposed `withStartOffset` / `withOffsetRetainer` on `DebeziumIO.Read` and the cross-language `ReadBuilder` ([#28248](https://github.com/apache/beam/issues/28248)).

## New Features / Improvements

Expand Down
40 changes: 40 additions & 0 deletions sdks/go/pkg/beam/io/xlang/debeziumio/debezium.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ type readFromDebeziumSchema struct {
MaxNumberOfRecords *int64
MaxTimeToRun *int64
ConnectionProperties []string
StartOffset []string
OffsetStoragePath *string
}

type debeziumConfig struct {
Expand Down Expand Up @@ -149,6 +151,44 @@ func ConnectionProperties(cp []string) readOption {
}
}

// StartOffset specifies the offset from which the connector should resume consuming
// changes. Each entry must be a "key=value" string, where numeric values are encoded
// as their decimal string representation.
//
// Example for PostgreSQL:
//
// debeziumio.StartOffset([]string{"lsn=28160840"})
//
// Example for MySQL:
//
// debeziumio.StartOffset([]string{"file=binlog.000001", "pos=156"})
//
// Obtain the offset from the output of a previous pipeline run. Numeric values such
// as LSN or binlog position are automatically parsed to Long on the Java side.
func StartOffset(offset []string) readOption {
return func(cfg *debeziumConfig) {
cfg.readSchema.StartOffset = offset
}
}

// OffsetStoragePath sets a path where the connector offset is automatically saved after each
// checkpoint and loaded on pipeline startup, allowing the pipeline to resume from where it
// left off without any manual offset management.
//
// The path can be on any filesystem supported by the active Beam runner
// (local disk, GCS, S3, etc.).
//
// Example:
//
// debeziumio.OffsetStoragePath("gs://my-bucket/debezium/orders-offset.json")
//
// When set, takes precedence over StartOffset.
func OffsetStoragePath(path string) readOption {
return func(cfg *debeziumConfig) {
cfg.readSchema.OffsetStoragePath = &path
}
}

// ExpansionAddr sets the expansion service address to use for DebeziumIO cross-langauage transform.
func ExpansionAddr(expansionAddr string) readOption {
return func(cfg *debeziumConfig) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,10 @@ public abstract static class Read<T> extends PTransform<PBegin, PCollection<T>>

abstract @Nullable Long getPollingTimeout();

abstract @Nullable Map<String, Object> getStartOffset();

abstract @Nullable OffsetRetainer getOffsetRetainer();

abstract @Nullable Coder<T> getCoder();

abstract Builder<T> toBuilder();
Expand All @@ -162,6 +166,10 @@ abstract static class Builder<T> {

abstract Builder<T> setPollingTimeout(Long miliseconds);

abstract Builder<T> setStartOffset(Map<String, Object> startOffset);

abstract Builder<T> setOffsetRetainer(OffsetRetainer retainer);

abstract Read<T> build();
}

Expand Down Expand Up @@ -230,6 +238,74 @@ public Read<T> withPollingTimeout(Long miliseconds) {
return toBuilder().setPollingTimeout(miliseconds).build();
}

/**
* Sets a starting offset so the connector resumes consuming changes from a previously seen
* position rather than from the beginning of the change stream.
*
* <p>The offset format is connector-specific. You can capture the current offset for each
* processed record inside your {@link SourceRecordMapper} via {@link
* org.apache.kafka.connect.source.SourceRecord#sourceOffset()} and persist it externally (for
* example in Cloud Storage, a database, or a local file). On the next pipeline run, pass the
* last saved offset here.
*
* <p>Example (PostgreSQL):
*
* <pre>{@code
* // Capture the offset inside the SourceRecordMapper:
* Map<String, Object> offset = sourceRecord.sourceOffset();
* // Persist 'offset' externally, then on restart:
* DebeziumIO.read()
* .withConnectorConfiguration(config)
* .withStartOffset(savedOffset)
* .withFormatFunction(myMapper);
* }</pre>
*
* @param startOffset A map representing the resumption point, as returned by {@code
* SourceRecord#sourceOffset()}.
* @return PTransform {@link #read}
*/
public Read<T> withStartOffset(Map<String, Object> startOffset) {
checkArgument(startOffset != null, "startOffset can not be null");
return toBuilder().setStartOffset(startOffset).build();
}

/**
* Sets an {@link OffsetRetainer} that automatically saves and restores the connector offset,
* allowing the pipeline to resume from where it left off after a restart without any manual
* offset management.
*
* <p>When a retainer is configured:
*
* <ol>
* <li>At pipeline startup, {@link OffsetRetainer#loadOffset()} is called. If a saved offset
* is found, the connector resumes from that position; otherwise it starts from the
* beginning of the change stream.
* <li>After each successful checkpoint ({@code task.commit()}), {@link
* OffsetRetainer#saveOffset(Map)} is called with the latest committed offset.
* </ol>
*
* <p>The built-in {@link FileSystemOffsetRetainer} persists the offset as a JSON file on any
* Beam-compatible filesystem (local, GCS, S3, etc.):
*
* <pre>{@code
* DebeziumIO.read()
* .withConnectorConfiguration(config)
* .withOffsetRetainer(
* new FileSystemOffsetRetainer("gs://my-bucket/debezium/orders-offset.json"))
* .withFormatFunction(myMapper);
* }</pre>
*
* <p>When both a retainer and {@link #withStartOffset(Map)} are set, the retainer takes
* precedence. Use {@link #withStartOffset(Map)} alone for a one-time manual override.
*
* @param retainer The {@link OffsetRetainer} to use for loading and saving offsets.
* @return PTransform {@link #read}
*/
public Read<T> withOffsetRetainer(OffsetRetainer retainer) {
checkArgument(retainer != null, "retainer can not be null");
return toBuilder().setOffsetRetainer(retainer).build();
}

protected Schema getRecordSchema() {
KafkaSourceConsumerFn<T> fn =
new KafkaSourceConsumerFn<>(getConnectorConfiguration().getConnectorClass().get(), this);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.io.debezium;

import com.google.auto.service.AutoService;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
Expand Down Expand Up @@ -78,6 +79,8 @@ public static class Configuration extends CrossLanguageConfiguration {
private @Nullable List<String> connectionProperties;
private @Nullable Long maxNumberOfRecords;
private @Nullable Long maxTimeToRun;
private @Nullable List<String> startOffset;
private @Nullable String offsetStoragePath;

public void setConnectionProperties(@Nullable List<String> connectionProperties) {
this.connectionProperties = connectionProperties;
Expand All @@ -90,6 +93,14 @@ public void setMaxNumberOfRecords(@Nullable Long maxNumberOfRecords) {
public void setMaxTimeToRun(@Nullable Long maxTimeToRun) {
this.maxTimeToRun = maxTimeToRun;
}

public void setStartOffset(@Nullable List<String> startOffset) {
this.startOffset = startOffset;
}

public void setOffsetStoragePath(@Nullable String offsetStoragePath) {
this.offsetStoragePath = offsetStoragePath;
}
}

@Override
Expand Down Expand Up @@ -123,6 +134,33 @@ public PTransform<PBegin, PCollection<String>> buildExternal(Configuration confi
readTransform = readTransform.withMaxTimeToRun(configuration.maxTimeToRun);
}

if (configuration.startOffset != null) {
Map<String, Object> startOffsetMap = new HashMap<>();
for (String property : configuration.startOffset) {
String[] parts = property.split("=", 2);
if (parts.length != 2) {
throw new IllegalArgumentException(
"Invalid startOffset entry: \""
+ property
+ "\". Expected format is \"key=value\".");
}
String key = parts[0];
String value = parts[1];
try {
startOffsetMap.put(key, Long.parseLong(value));
} catch (NumberFormatException e) {
startOffsetMap.put(key, value);
}
Comment thread
tkaymak marked this conversation as resolved.
}
readTransform = readTransform.withStartOffset(startOffsetMap);
}

if (configuration.offsetStoragePath != null) {
readTransform =
readTransform.withOffsetRetainer(
FileSystemOffsetRetainer.of(configuration.offsetStoragePath));
}

return readTransform;
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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.
*/
package org.apache.beam.io.debezium;

import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Collections;
import java.util.Map;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* An {@link OffsetRetainer} that persists the Debezium connector offset as a JSON file using Beam's
* {@link FileSystems} abstraction.
*
* <p>The {@code path} argument can point to any filesystem supported by the active Beam runner,
* including local disk, Google Cloud Storage, Amazon S3, and others
*
* <p>On every {@code task.commit()}, the latest offset is serialised to JSON and written to the
* given path (overwriting the previous file). On pipeline startup the file is read back and the
* connector resumes from the stored position. If the file does not yet exist the connector starts
* from the beginning of the change stream.
*
* <p>Example — resume from GCS:
*
* <pre>{@code
* DebeziumIO.read()
* .withConnectorConfiguration(config)
* .withOffsetRetainer(
* FileSystemOffsetRetainer.of("gs://my-bucket/debezium/orders-offset.json"))
* .withFormatFunction(myMapper);
* }</pre>
*
* <p>Example — local filesystem (useful for testing):
*
* <pre>{@code
* DebeziumIO.read()
* .withConnectorConfiguration(config)
* .withOffsetRetainer(FileSystemOffsetRetainer.of("/tmp/debezium-offset.json"))
* .withFormatFunction(myMapper);
* }</pre>
*
* <p><b>Note:</b> writes are performed atomically: the offset is first written to a {@code .tmp}
* sibling file and then renamed to the final path, so a mid-write crash leaves the previous offset
* intact.
*/
public class FileSystemOffsetRetainer implements OffsetRetainer {

private static final Logger LOG = LoggerFactory.getLogger(FileSystemOffsetRetainer.class);
private static final TypeReference<Map<String, Object>> MAP_TYPE = new TypeReference<>() {};

private final String path;

// ObjectMapper is thread-safe after configuration and does not need to be serialised.
private transient @Nullable ObjectMapper objectMapper;

// Tracks the last successfully saved offset so repeated identical saves are skipped.
private transient @Nullable Map<String, Object> lastSavedOffset;

private FileSystemOffsetRetainer(String path) {
this.path = path;
}

/** Creates a new {@code FileSystemOffsetRetainer} that stores the offset at {@code path}. */
public static FileSystemOffsetRetainer of(String path) {
return new FileSystemOffsetRetainer(path);
}

private ObjectMapper mapper() {
if (objectMapper == null) {
objectMapper = new ObjectMapper();
}
return objectMapper;
}

/**
* Reads the offset JSON file and returns its contents, or {@code null} if the file does not yet
* exist (first run). Throws {@link RuntimeException} if the file exists but cannot be read, to
* prevent silently reprocessing data from the beginning.
*/
@Override
public @Nullable Map<String, Object> loadOffset() {
try {
ResourceId resourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false);
try (ReadableByteChannel channel = FileSystems.open(resourceId);
InputStream stream = Channels.newInputStream(channel)) {
Map<String, Object> offset = mapper().readValue(stream, MAP_TYPE);
LOG.info("OffsetRetainer: loaded offset from {}: {}", path, offset);
return offset;
}
} catch (FileNotFoundException e) {
LOG.info("OffsetRetainer: no offset file found at {}; starting from the beginning.", path);
return null;
} catch (IOException e) {
throw new RuntimeException(
"OffsetRetainer: failed to read offset from "
+ path
+ ". "
+ "Delete the file to restart from the beginning.",
e);
}
}

/**
* Serialises {@code offset} to JSON and writes it atomically to the configured path.
*
* <p>If the offset is identical to the last successfully written one, the write is skipped to
* avoid unnecessary I/O on every checkpoint.
*
* <p>Otherwise the data is first written to a {@code .tmp} sibling file and then renamed to the
* final path, so a mid-write crash leaves the previous offset intact.
*
* <p>Errors are logged as warnings and swallowed so the pipeline continues.
*/
@Override
public void saveOffset(Map<String, Object> offset) {
if (offset.equals(lastSavedOffset)) {
LOG.debug("OffsetRetainer: offset unchanged, skipping write to {}", path);
return;
}
String tmpPath = path + ".tmp";
try {
ResourceId tmpResourceId = FileSystems.matchNewResource(tmpPath, /* isDirectory= */ false);
try (WritableByteChannel channel = FileSystems.create(tmpResourceId, "application/json");
OutputStream stream = Channels.newOutputStream(channel)) {
mapper().writeValue(stream, offset);
}
ResourceId finalResourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false);
FileSystems.rename(
Collections.singletonList(tmpResourceId), Collections.singletonList(finalResourceId));
lastSavedOffset = offset;
LOG.debug("OffsetRetainer: saved offset to {}: {}", path, offset);
} catch (IOException e) {
LOG.warn(
"OffsetRetainer: failed to save offset to {}."
+ " The offset will be lost if the pipeline restarts.",
path,
e);
}
}
}
Loading
Loading