Abacn commented on code in PR #37750: URL: https://github.com/apache/beam/pull/37750#discussion_r2914285367
########## sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/FileSystemOffsetRetainer.java: ########## @@ -0,0 +1,136 @@ +/* + * 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.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( + * new FileSystemOffsetRetainer("gs://my-bucket/debezium/orders-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p>Example — local filesystem (useful for testing): + * + * <pre>{@code + * DebeziumIO.read() + * .withConnectorConfiguration(config) + * .withOffsetRetainer(new FileSystemOffsetRetainer("/tmp/debezium-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p><b>Note:</b> writes are not atomic. If the pipeline is killed mid-write, the offset file may + * be corrupt. In that case, delete the file and the connector will restart from the beginning. + */ +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; + + public FileSystemOffsetRetainer(String path) { + this.path = 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 exist + * or cannot be read. + */ + @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) { + LOG.warn( + "OffsetRetainer: failed to load offset from {}; starting from the beginning.", path, e); + return null; + } + } + + /** + * Serialises {@code offset} to JSON and writes it to the configured path, overwriting any + * existing file. Errors are logged as warnings and swallowed so the pipeline continues. + */ + @Override + public void saveOffset(Map<String, Object> offset) { + try { + ResourceId resourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false); Review Comment: Should we write to a tmp file (e.g. `final/file.path.tmp`) then move to final path, in case pipeline gets crashed in the middle, then we get corrupted file and previous written offset is lost ########## sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java: ########## @@ -122,7 +122,21 @@ public class KafkaSourceConsumerFn<T> extends DoFn<Map<String, String>, T> { @GetInitialRestriction public OffsetHolder getInitialRestriction(@Element Map<String, String> unused) throws IOException { - return new OffsetHolder(null, null, null, spec.getMaxNumberOfRecords(), spec.getMaxTimeToRun()); + Map<String, Object> initialOffset = null; + + // Retainer takes precedence: it reflects the most recently committed position. + OffsetRetainer retainer = spec.getOffsetRetainer(); + if (retainer != null) { + initialOffset = retainer.loadOffset(); + } + + // Fall back to the explicit one-time override when the retainer has no saved offset. Review Comment: +1 this precedence choice makes sense. ########## sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/FileSystemOffsetRetainer.java: ########## @@ -0,0 +1,136 @@ +/* + * 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.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( + * new FileSystemOffsetRetainer("gs://my-bucket/debezium/orders-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p>Example — local filesystem (useful for testing): + * + * <pre>{@code + * DebeziumIO.read() + * .withConnectorConfiguration(config) + * .withOffsetRetainer(new FileSystemOffsetRetainer("/tmp/debezium-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p><b>Note:</b> writes are not atomic. If the pipeline is killed mid-write, the offset file may + * be corrupt. In that case, delete the file and the connector will restart from the beginning. + */ +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; + + public FileSystemOffsetRetainer(String path) { + this.path = 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 exist + * or cannot be read. + */ + @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) { + LOG.warn( Review Comment: It suggests the path cannot be read. I wonder if we should surface the Exception and fail in this case. ########## sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/FileSystemOffsetRetainer.java: ########## @@ -0,0 +1,136 @@ +/* + * 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.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( + * new FileSystemOffsetRetainer("gs://my-bucket/debezium/orders-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p>Example — local filesystem (useful for testing): + * + * <pre>{@code + * DebeziumIO.read() + * .withConnectorConfiguration(config) + * .withOffsetRetainer(new FileSystemOffsetRetainer("/tmp/debezium-offset.json")) + * .withFormatFunction(myMapper); + * }</pre> + * + * <p><b>Note:</b> writes are not atomic. If the pipeline is killed mid-write, the offset file may + * be corrupt. In that case, delete the file and the connector will restart from the beginning. + */ +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; + + public FileSystemOffsetRetainer(String path) { + this.path = 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 exist + * or cannot be read. + */ + @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) { + LOG.warn( + "OffsetRetainer: failed to load offset from {}; starting from the beginning.", path, e); + return null; + } + } + + /** + * Serialises {@code offset} to JSON and writes it to the configured path, overwriting any + * existing file. Errors are logged as warnings and swallowed so the pipeline continues. + */ + @Override + public void saveOffset(Map<String, Object> offset) { + try { + ResourceId resourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false); + try (WritableByteChannel channel = FileSystems.create(resourceId, "application/json"); Review Comment: Not sure how frequent a checkpoint gets issued, i.e. if we'll encounter similar issue like #37510. For now I wonder if we can compare the offset going to be saved with the last saved one, and only commit when it is modified, like what #37510 did -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
