Skip to content

Commit a5bc4a9

Browse files
subodh1810gl-pix
authored andcommitted
throw exception if we close engine before snapshot is complete + increase timeout for subsequent records (#4730)
* throw exception if we close engine before snapshot is complete + increase timeout for subsequent records * add comment + bump postgres version to use new changes
1 parent af9db0b commit a5bc4a9

File tree

4 files changed

+43
-8
lines changed

4 files changed

+43
-8
lines changed

airbyte-config/init/src/main/resources/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@
22
"sourceDefinitionId": "decd338e-5647-4c0b-adf4-da0e75f5a750",
33
"name": "Postgres",
44
"dockerRepository": "airbyte/source-postgres",
5-
"dockerImageTag": "0.3.6",
5+
"dockerImageTag": "0.3.7",
66
"documentationUrl": "https://hub.docker.com/r/airbyte/source-postgres",
77
"icon": "postgresql.svg"
88
}

airbyte-config/init/src/main/resources/seed/source_definitions.yaml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@
4646
- sourceDefinitionId: decd338e-5647-4c0b-adf4-da0e75f5a750
4747
name: Postgres
4848
dockerRepository: airbyte/source-postgres
49-
dockerImageTag: 0.3.6
49+
dockerImageTag: 0.3.7
5050
documentationUrl: https://hub.docker.com/r/airbyte/source-postgres
5151
icon: postgresql.svg
5252
- sourceDefinitionId: 9fa5862c-da7c-11eb-8d19-0242ac130003

airbyte-integrations/bases/debezium/src/main/java/io/airbyte/integrations/debezium/internals/DebeziumRecordIterator.java

Lines changed: 40 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424

2525
package io.airbyte.integrations.debezium.internals;
2626

27+
import com.fasterxml.jackson.databind.JsonNode;
2728
import com.google.common.collect.AbstractIterator;
2829
import io.airbyte.commons.concurrency.VoidCallable;
2930
import io.airbyte.commons.json.Jsons;
@@ -54,13 +55,14 @@ public class DebeziumRecordIterator extends AbstractIterator<ChangeEvent<String,
5455
private static final Logger LOGGER = LoggerFactory.getLogger(DebeziumRecordIterator.class);
5556

5657
private static final WaitTime FIRST_RECORD_WAIT_TIME_MINUTES = new WaitTime(5, TimeUnit.MINUTES);
57-
private static final WaitTime SUBSEQUENT_RECORD_WAIT_TIME_SECONDS = new WaitTime(5, TimeUnit.SECONDS);
58+
private static final WaitTime SUBSEQUENT_RECORD_WAIT_TIME_SECONDS = new WaitTime(1, TimeUnit.MINUTES);
5859

5960
private final LinkedBlockingQueue<ChangeEvent<String, String>> queue;
6061
private final CdcTargetPosition targetPosition;
6162
private final Supplier<Boolean> publisherStatusSupplier;
6263
private final VoidCallable requestClose;
6364
private boolean receivedFirstRecord;
65+
private boolean hasSnapshotFinished;
6466

6567
public DebeziumRecordIterator(LinkedBlockingQueue<ChangeEvent<String, String>> queue,
6668
CdcTargetPosition targetPosition,
@@ -71,6 +73,7 @@ public DebeziumRecordIterator(LinkedBlockingQueue<ChangeEvent<String, String>> q
7173
this.publisherStatusSupplier = publisherStatusSupplier;
7274
this.requestClose = requestClose;
7375
this.receivedFirstRecord = false;
76+
this.hasSnapshotFinished = true;
7477
}
7578

7679
@Override
@@ -90,13 +93,17 @@ protected ChangeEvent<String, String> computeNext() {
9093
// if within the timeout, the consumer could not get a record, it is time to tell the producer to
9194
// shutdown.
9295
if (next == null) {
96+
LOGGER.info("Closing cause next is returned as null");
9397
requestClose();
9498
LOGGER.info("no record found. polling again.");
9599
continue;
96100
}
97101

102+
JsonNode eventAsJson = Jsons.deserialize(next.value());
103+
hasSnapshotFinished = hasSnapshotFinished(eventAsJson);
104+
98105
// if the last record matches the target file position, it is time to tell the producer to shutdown.
99-
if (shouldSignalClose(next)) {
106+
if (shouldSignalClose(eventAsJson)) {
100107
requestClose();
101108
}
102109
receivedFirstRecord = true;
@@ -105,14 +112,35 @@ protected ChangeEvent<String, String> computeNext() {
105112
return endOfData();
106113
}
107114

115+
private boolean hasSnapshotFinished(JsonNode eventAsJson) {
116+
SnapshotMetadata snapshot = SnapshotMetadata.valueOf(eventAsJson.get("source").get("snapshot").asText().toUpperCase());
117+
return SnapshotMetadata.TRUE != snapshot;
118+
}
119+
120+
/**
121+
* Debezium was built as an ever running process which keeps on listening for new changes on DB and
122+
* immediately processing them. Airbyte needs debezium to work as a start stop mechanism. In order
123+
* to determine when to stop debezium engine we rely on few factors 1. TargetPosition logic. At the
124+
* beginning of the sync we define a target position in the logs of the DB. This can be an LSN or
125+
* anything specific to the DB which can help us identify that we have reached a specific position
126+
* in the log based replication When we start processing records from debezium, we extract the the
127+
* log position from the metadata of the record and compare it with our target that we defined at
128+
* the beginning of the sync. If we have reached the target position, we shutdown the debezium
129+
* engine 2. The TargetPosition logic might not always work and in order to tackle that we have
130+
* another logic where if we do not receive records from debezium for a given duration, we ask
131+
* debezium engine to shutdown 3. We also take the Snapshot into consideration, when a connector is
132+
* running for the first time, we let it complete the snapshot and only after the completion of
133+
* snapshot we should shutdown the engine. If we are closing the engine before completion of
134+
* snapshot, we throw an exception
135+
*/
108136
@Override
109137
public void close() throws Exception {
110138
requestClose.call();
139+
throwExceptionIfSnapshotNotFinished();
111140
}
112141

113-
private boolean shouldSignalClose(ChangeEvent<String, String> event) {
114-
115-
return targetPosition.reachedTargetPosition(Jsons.deserialize(event.value()));
142+
private boolean shouldSignalClose(JsonNode eventAsJson) {
143+
return targetPosition.reachedTargetPosition(eventAsJson);
116144
}
117145

118146
private void requestClose() {
@@ -121,6 +149,13 @@ private void requestClose() {
121149
} catch (Exception e) {
122150
throw new RuntimeException(e);
123151
}
152+
throwExceptionIfSnapshotNotFinished();
153+
}
154+
155+
private void throwExceptionIfSnapshotNotFinished() {
156+
if (!hasSnapshotFinished) {
157+
throw new RuntimeException("Closing down debezium engine but snapshot has not finished");
158+
}
124159
}
125160

126161
private static class WaitTime {

airbyte-integrations/connectors/source-postgres/Dockerfile

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,5 +8,5 @@ COPY build/distributions/${APPLICATION}*.tar ${APPLICATION}.tar
88

99
RUN tar xf ${APPLICATION}.tar --strip-components=1
1010

11-
LABEL io.airbyte.version=0.3.6
11+
LABEL io.airbyte.version=0.3.7
1212
LABEL io.airbyte.name=airbyte/source-postgres

0 commit comments

Comments
 (0)