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
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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.flink.cdc.connectors.base;

import org.apache.flink.cdc.common.annotation.PublicEvolving;
import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset;
import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase;
import org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkKind;

import java.util.Map;

/** CDC event dispatcher which provides ability to dispatch watermark. */
@PublicEvolving
public interface WatermarkDispatcher {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add an annotation for this API will be better.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


void dispatchWatermarkEvent(
Map<String, ?> sourcePartition,
SourceSplitBase sourceSplit,
Offset watermark,
WatermarkKind watermarkKind)
throws InterruptedException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.cdc.connectors.base.relational;

import org.apache.flink.cdc.connectors.base.WatermarkDispatcher;
import org.apache.flink.cdc.connectors.base.relational.handler.SchemaChangeEventHandler;
import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset;
import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase;
Expand Down Expand Up @@ -61,7 +62,8 @@
* this is useful for downstream to deserialize the {@link HistoryRecord} back.
* </pre>
*/
public class JdbcSourceEventDispatcher<P extends Partition> extends EventDispatcher<P, TableId> {
public class JdbcSourceEventDispatcher<P extends Partition> extends EventDispatcher<P, TableId>
implements WatermarkDispatcher {
private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceEventDispatcher.class);

public static final String HISTORY_RECORD_FIELD = "historyRecord";
Expand Down Expand Up @@ -238,6 +240,7 @@ public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedExcepti
}
}

@Override
public void dispatchWatermarkEvent(
Map<String, ?> sourcePartition,
SourceSplitBase sourceSplit,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ protected void dispatchLowWaterMarkEvent(
Context context, SourceSplitBase split, Offset lowWatermark) throws Exception {
if (context instanceof JdbcSourceFetchTaskContext) {
((JdbcSourceFetchTaskContext) context)
.getDispatcher()
.getWaterMarkDispatcher()
.dispatchWatermarkEvent(
((JdbcSourceFetchTaskContext) context)
.getPartition()
Expand All @@ -157,7 +157,7 @@ protected void dispatchHighWaterMarkEvent(
Context context, SourceSplitBase split, Offset highWatermark) throws Exception {
if (context instanceof JdbcSourceFetchTaskContext) {
((JdbcSourceFetchTaskContext) context)
.getDispatcher()
.getWaterMarkDispatcher()
.dispatchWatermarkEvent(
((JdbcSourceFetchTaskContext) context)
.getPartition()
Expand All @@ -180,7 +180,7 @@ protected void dispatchEndWaterMarkEvent(
Context context, SourceSplitBase split, Offset endWatermark) throws Exception {
if (context instanceof JdbcSourceFetchTaskContext) {
((JdbcSourceFetchTaskContext) context)
.getDispatcher()
.getWaterMarkDispatcher()
.dispatchWatermarkEvent(
((JdbcSourceFetchTaskContext) context)
.getPartition()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,18 @@

package org.apache.flink.cdc.connectors.base.source.reader.external;

import org.apache.flink.annotation.Internal;
import org.apache.flink.cdc.connectors.base.WatermarkDispatcher;
import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
import org.apache.flink.cdc.connectors.base.config.SourceConfig;
import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils;
import org.apache.flink.table.types.logical.RowType;

import io.debezium.config.CommonConnectorConfig;
import io.debezium.data.Envelope;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Partition;
import io.debezium.relational.RelationalDatabaseSchema;
Expand All @@ -43,6 +45,7 @@
import java.util.stream.Collectors;

/** The context for fetch task that fetching data of snapshot split from JDBC data source. */
@Internal
public abstract class JdbcSourceFetchTaskContext implements FetchTask.Context {

protected final JdbcSourceConfig sourceConfig;
Expand Down Expand Up @@ -171,7 +174,9 @@ public SchemaNameAdjuster getSchemaNameAdjuster() {

public abstract ErrorHandler getErrorHandler();

public abstract JdbcSourceEventDispatcher getDispatcher();
public abstract EventDispatcher getEventDispatcher();

public abstract WatermarkDispatcher getWaterMarkDispatcher();

public abstract OffsetContext getOffsetContext();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.cdc.connectors.db2.source.fetch;

import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit;
import org.apache.flink.cdc.connectors.base.source.reader.external.AbstractScanFetchTask;
Expand Down Expand Up @@ -74,7 +73,7 @@ protected void executeDataSnapshot(Context context) throws Exception {
sourceFetchContext.getSnapshotChangeEventSourceMetrics(),
sourceFetchContext.getDatabaseSchema(),
sourceFetchContext.getConnection(),
sourceFetchContext.getDispatcher(),
sourceFetchContext.getEventDispatcher(),
sourceFetchContext.getSnapshotReceiver(),
snapshotSplit);
Db2SnapshotSplitChangeEventSourceContext changeEventSourceContext =
Expand Down Expand Up @@ -132,7 +131,8 @@ private StreamSplitReadTask createBackFillLsnSplitReadTask(
new Db2ConnectorConfig(dezConf),
context.getConnection(),
context.getMetaDataConnection(),
context.getDispatcher(),
context.getEventDispatcher(),
context.getWaterMarkDispatcher(),
context.getErrorHandler(),
context.getDatabaseSchema(),
backfillBinlogSplit);
Expand All @@ -150,7 +150,7 @@ public static class Db2SnapshotSplitReadTask
private final Db2ConnectorConfig connectorConfig;
private final Db2DatabaseSchema databaseSchema;
private final Db2Connection jdbcConnection;
private final JdbcSourceEventDispatcher<Db2Partition> dispatcher;
private final EventDispatcher<Db2Partition, TableId> dispatcher;
private final Clock clock;
private final SnapshotSplit snapshotSplit;
private final Db2OffsetContext offsetContext;
Expand All @@ -163,7 +163,7 @@ public Db2SnapshotSplitReadTask(
SnapshotProgressListener<Db2Partition> snapshotProgressListener,
Db2DatabaseSchema databaseSchema,
Db2Connection jdbcConnection,
JdbcSourceEventDispatcher<Db2Partition> dispatcher,
EventDispatcher<Db2Partition, TableId> dispatcher,
EventDispatcher.SnapshotReceiver<Db2Partition> snapshotReceiver,
SnapshotSplit snapshotSplit) {
super(connectorConfig, snapshotProgressListener);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.cdc.connectors.db2.source.fetch;

import org.apache.flink.cdc.connectors.base.WatermarkDispatcher;
import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import org.apache.flink.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory;
Expand Down Expand Up @@ -216,7 +217,12 @@ public Db2SourceConfig getSourceConfig() {
}

@Override
public JdbcSourceEventDispatcher<Db2Partition> getDispatcher() {
public JdbcSourceEventDispatcher getEventDispatcher() {
return dispatcher;
}

@Override
public WatermarkDispatcher getWaterMarkDispatcher() {
return dispatcher;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.cdc.connectors.db2.source.fetch;

import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import org.apache.flink.cdc.connectors.base.WatermarkDispatcher;
import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset;
import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase;
import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit;
Expand All @@ -34,7 +34,9 @@
import io.debezium.connector.db2.Db2StreamingChangeEventSource;
import io.debezium.connector.db2.Lsn;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
import io.debezium.relational.TableId;
import io.debezium.util.Clock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -61,7 +63,8 @@ public void execute(Context context) throws Exception {
sourceFetchContext.getDbzConnectorConfig(),
sourceFetchContext.getConnection(),
sourceFetchContext.getMetaDataConnection(),
sourceFetchContext.getDispatcher(),
sourceFetchContext.getEventDispatcher(),
sourceFetchContext.getWaterMarkDispatcher(),
sourceFetchContext.getErrorHandler(),
sourceFetchContext.getDatabaseSchema(),
split);
Expand Down Expand Up @@ -96,28 +99,29 @@ public static class StreamSplitReadTask extends Db2StreamingChangeEventSource {

private static final Logger LOG = LoggerFactory.getLogger(StreamSplitReadTask.class);
private final StreamSplit lsnSplit;
private final JdbcSourceEventDispatcher<Db2Partition> dispatcher;
private final WatermarkDispatcher watermarkDispatcher;
private final ErrorHandler errorHandler;
private ChangeEventSourceContext context;

public StreamSplitReadTask(
Db2ConnectorConfig connectorConfig,
Db2Connection connection,
Db2Connection metadataConnection,
JdbcSourceEventDispatcher<Db2Partition> dispatcher,
EventDispatcher<Db2Partition, TableId> eventDispatcher,
WatermarkDispatcher watermarkDispatcher,
ErrorHandler errorHandler,
Db2DatabaseSchema schema,
StreamSplit lsnSplit) {
super(
connectorConfig,
connection,
metadataConnection,
dispatcher,
eventDispatcher,
errorHandler,
Clock.system(),
schema);
this.lsnSplit = lsnSplit;
this.dispatcher = dispatcher;
this.watermarkDispatcher = watermarkDispatcher;
this.errorHandler = errorHandler;
}

Expand All @@ -130,7 +134,7 @@ public void afterHandleLsn(Db2Partition partition, Lsn toLsn) {
if (currentLsnOffset.isAtOrAfter(endingOffset)) {
// send streaming end event
try {
dispatcher.dispatchWatermarkEvent(
watermarkDispatcher.dispatchWatermarkEvent(
partition.getSourcePartition(),
lsnSplit,
currentLsnOffset,
Expand Down
Loading
Loading