|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.io.debezium; |
| 19 | + |
| 20 | +import com.fasterxml.jackson.core.type.TypeReference; |
| 21 | +import com.fasterxml.jackson.databind.ObjectMapper; |
| 22 | +import java.io.FileNotFoundException; |
| 23 | +import java.io.IOException; |
| 24 | +import java.io.InputStream; |
| 25 | +import java.io.OutputStream; |
| 26 | +import java.nio.channels.Channels; |
| 27 | +import java.nio.channels.ReadableByteChannel; |
| 28 | +import java.nio.channels.WritableByteChannel; |
| 29 | +import java.util.Map; |
| 30 | +import org.apache.beam.sdk.io.FileSystems; |
| 31 | +import org.apache.beam.sdk.io.fs.ResourceId; |
| 32 | +import org.checkerframework.checker.nullness.qual.Nullable; |
| 33 | +import org.slf4j.Logger; |
| 34 | +import org.slf4j.LoggerFactory; |
| 35 | + |
| 36 | +/** |
| 37 | + * An {@link OffsetRetainer} that persists the Debezium connector offset as a JSON file using Beam's |
| 38 | + * {@link FileSystems} abstraction. |
| 39 | + * |
| 40 | + * <p>The {@code path} argument can point to any filesystem supported by the active Beam runner, |
| 41 | + * including local disk, Google Cloud Storage, Amazon S3, and others |
| 42 | + * |
| 43 | + * <p>On every {@code task.commit()}, the latest offset is serialised to JSON and written to the |
| 44 | + * given path (overwriting the previous file). On pipeline startup the file is read back and the |
| 45 | + * connector resumes from the stored position. If the file does not yet exist the connector starts |
| 46 | + * from the beginning of the change stream. |
| 47 | + * |
| 48 | + * <p>Example — resume from GCS: |
| 49 | + * |
| 50 | + * <pre>{@code |
| 51 | + * DebeziumIO.read() |
| 52 | + * .withConnectorConfiguration(config) |
| 53 | + * .withOffsetRetainer( |
| 54 | + * new FileSystemOffsetRetainer("gs://my-bucket/debezium/orders-offset.json")) |
| 55 | + * .withFormatFunction(myMapper); |
| 56 | + * }</pre> |
| 57 | + * |
| 58 | + * <p>Example — local filesystem (useful for testing): |
| 59 | + * |
| 60 | + * <pre>{@code |
| 61 | + * DebeziumIO.read() |
| 62 | + * .withConnectorConfiguration(config) |
| 63 | + * .withOffsetRetainer(new FileSystemOffsetRetainer("/tmp/debezium-offset.json")) |
| 64 | + * .withFormatFunction(myMapper); |
| 65 | + * }</pre> |
| 66 | + * |
| 67 | + * <p><b>Note:</b> writes are not atomic. If the pipeline is killed mid-write, the offset file may |
| 68 | + * be corrupt. In that case, delete the file and the connector will restart from the beginning. |
| 69 | + */ |
| 70 | +public class FileSystemOffsetRetainer implements OffsetRetainer { |
| 71 | + |
| 72 | + private static final Logger LOG = LoggerFactory.getLogger(FileSystemOffsetRetainer.class); |
| 73 | + private static final TypeReference<Map<String, Object>> MAP_TYPE = new TypeReference<>() {}; |
| 74 | + |
| 75 | + private final String path; |
| 76 | + |
| 77 | + // ObjectMapper is thread-safe after configuration and does not need to be serialised. |
| 78 | + private transient @Nullable ObjectMapper objectMapper; |
| 79 | + |
| 80 | + public FileSystemOffsetRetainer(String path) { |
| 81 | + this.path = path; |
| 82 | + } |
| 83 | + |
| 84 | + private ObjectMapper mapper() { |
| 85 | + if (objectMapper == null) { |
| 86 | + objectMapper = new ObjectMapper(); |
| 87 | + } |
| 88 | + return objectMapper; |
| 89 | + } |
| 90 | + |
| 91 | + /** |
| 92 | + * Reads the offset JSON file and returns its contents, or {@code null} if the file does not exist |
| 93 | + * or cannot be read. |
| 94 | + */ |
| 95 | + @Override |
| 96 | + public @Nullable Map<String, Object> loadOffset() { |
| 97 | + try { |
| 98 | + ResourceId resourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false); |
| 99 | + try (ReadableByteChannel channel = FileSystems.open(resourceId); |
| 100 | + InputStream stream = Channels.newInputStream(channel)) { |
| 101 | + Map<String, Object> offset = mapper().readValue(stream, MAP_TYPE); |
| 102 | + LOG.info("OffsetRetainer: loaded offset from {}: {}", path, offset); |
| 103 | + return offset; |
| 104 | + } |
| 105 | + } catch (FileNotFoundException e) { |
| 106 | + LOG.info("OffsetRetainer: no offset file found at {}; starting from the beginning.", path); |
| 107 | + return null; |
| 108 | + } catch (IOException e) { |
| 109 | + LOG.warn( |
| 110 | + "OffsetRetainer: failed to load offset from {}; starting from the beginning.", path, e); |
| 111 | + return null; |
| 112 | + } |
| 113 | + } |
| 114 | + |
| 115 | + /** |
| 116 | + * Serialises {@code offset} to JSON and writes it to the configured path, overwriting any |
| 117 | + * existing file. Errors are logged as warnings and swallowed so the pipeline continues. |
| 118 | + */ |
| 119 | + @Override |
| 120 | + public void saveOffset(Map<String, Object> offset) { |
| 121 | + try { |
| 122 | + ResourceId resourceId = FileSystems.matchNewResource(path, /* isDirectory= */ false); |
| 123 | + try (WritableByteChannel channel = FileSystems.create(resourceId, "application/json"); |
| 124 | + OutputStream stream = Channels.newOutputStream(channel)) { |
| 125 | + mapper().writeValue(stream, offset); |
| 126 | + } |
| 127 | + LOG.debug("OffsetRetainer: saved offset to {}: {}", path, offset); |
| 128 | + } catch (IOException e) { |
| 129 | + LOG.warn( |
| 130 | + "OffsetRetainer: failed to save offset to {}." |
| 131 | + + " The offset will be lost if the pipeline restarts.", |
| 132 | + path, |
| 133 | + e); |
| 134 | + } |
| 135 | + } |
| 136 | +} |
0 commit comments