Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[oracle] Correct the naming error #2405

Merged
merged 3 commits into from
Aug 26, 2023
Merged
Show file tree
Hide file tree
Changes from 2 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
Expand Up @@ -110,34 +110,34 @@ public void execute(Context context) throws Exception {
sourceFetchContext.getPartition(),
sourceFetchContext.getOffsetContext());

final StreamSplit backfillBinlogSplit =
final StreamSplit backfillRedoLogSplit =
createBackfillRedoLogSplit(changeEventSourceContext);
// optimization that skip the binlog read when the low watermark equals high
// optimization that skip the redo log read when the low watermark equals high
// watermark
final boolean binlogBackfillRequired =
backfillBinlogSplit
final boolean redoLogBackfillRequired =
backfillRedoLogSplit
.getEndingOffset()
.isAfter(backfillBinlogSplit.getStartingOffset());
if (!binlogBackfillRequired) {
dispatchBinlogEndEvent(
backfillBinlogSplit,
.isAfter(backfillRedoLogSplit.getStartingOffset());
if (!redoLogBackfillRequired) {
dispatchRedoLogEndEvent(
backfillRedoLogSplit,
sourceFetchContext.getPartition().getSourcePartition(),
((OracleSourceFetchTaskContext) context).getDispatcher());
taskRunning = false;
return;
}
// execute redoLog read task
if (snapshotResult.isCompletedOrSkipped()) {
final RedoLogSplitReadTask backfillBinlogReadTask =
createBackfillRedoLogReadTask(backfillBinlogSplit, sourceFetchContext);
final RedoLogSplitReadTask backfillRedoLogReadTask =
createBackfillRedoLogReadTask(backfillRedoLogSplit, sourceFetchContext);

final LogMinerOracleOffsetContextLoader loader =
new LogMinerOracleOffsetContextLoader(
((OracleSourceFetchTaskContext) context).getDbzConnectorConfig());
final OracleOffsetContext oracleOffsetContext =
loader.load(backfillBinlogSplit.getStartingOffset().getOffset());
backfillBinlogReadTask.execute(
new SnapshotBinlogSplitChangeEventSourceContext(),
loader.load(backfillRedoLogSplit.getStartingOffset().getOffset());
backfillRedoLogReadTask.execute(
new SnapshotRedoLogSplitChangeEventSourceContext(),
sourceFetchContext.getPartition(),
oracleOffsetContext);
taskRunning = false;
Expand All @@ -160,7 +160,7 @@ private StreamSplit createBackfillRedoLogSplit(
}

private RedoLogSplitReadTask createBackfillRedoLogReadTask(
StreamSplit backfillBinlogSplit, OracleSourceFetchTaskContext context) {
StreamSplit backfillRedoLogSplit, OracleSourceFetchTaskContext context) {
// we should only capture events for the current table,
// otherwise, we may can't find corresponding schema
Configuration dezConf =
Expand All @@ -171,7 +171,7 @@ private RedoLogSplitReadTask createBackfillRedoLogReadTask(
// Disable heartbeat event in snapshot split fetcher
.with(Heartbeat.HEARTBEAT_INTERVAL, 0)
.build();
// task to read binlog and backfill for current split
// task to read redo log and backfill for current split
return new RedoLogSplitReadTask(
new OracleConnectorConfig(dezConf),
context.getConnection(),
Expand All @@ -180,18 +180,18 @@ private RedoLogSplitReadTask createBackfillRedoLogReadTask(
context.getDatabaseSchema(),
context.getSourceConfig().getOriginDbzConnectorConfig(),
context.getStreamingChangeEventSourceMetrics(),
backfillBinlogSplit);
backfillRedoLogSplit);
}

private void dispatchBinlogEndEvent(
StreamSplit backFillBinlogSplit,
private void dispatchRedoLogEndEvent(
StreamSplit backFillRedoLogSplit,
Map<String, ?> sourcePartition,
JdbcSourceEventDispatcher<OraclePartition> eventDispatcher)
throws InterruptedException {
eventDispatcher.dispatchWatermarkEvent(
sourcePartition,
backFillBinlogSplit,
backFillBinlogSplit.getEndingOffset(),
backFillRedoLogSplit,
backFillRedoLogSplit.getEndingOffset(),
WatermarkKind.END);
}

Expand Down Expand Up @@ -446,10 +446,10 @@ public boolean isRunning() {
}

/**
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded binlog task
* of a snapshot split task.
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded redo log
* task of a snapshot split task.
*/
public class SnapshotBinlogSplitChangeEventSourceContext
public class SnapshotRedoLogSplitChangeEventSourceContext
implements ChangeEventSource.ChangeEventSourceContext {

public void finished() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,8 @@ public void close() {
}

/**
* A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark
* to highWatermark) binlog.
* A wrapped task to read all redo log for table and also supports read bounded (from
* lowWatermark to highWatermark) redo log.
*/
public static class RedoLogSplitReadTask extends LogMinerStreamingChangeEventSource {

Expand Down Expand Up @@ -138,13 +138,13 @@ public void execute(
protected void afterHandleScn(
OraclePartition partition, OracleOffsetContext offsetContext) {
super.afterHandleScn(partition, offsetContext);
// check do we need to stop for fetch binlog for snapshot split.
// check do we need to stop for fetch redo log for snapshot split.
if (isBoundedRead()) {
final RedoLogOffset currentRedoLogOffset =
getCurrentRedoLogOffset(offsetContext.getOffset());
// reach the high watermark, the binlog fetcher should be finished
// reach the high watermark, the redo log fetcher should be finished
if (currentRedoLogOffset.isAtOrAfter(redoLogSplit.getEndingOffset())) {
// send binlog end event
// send redo log end event
try {
dispatcher.dispatchWatermarkEvent(
partition.getSourcePartition(),
Expand All @@ -154,10 +154,10 @@ protected void afterHandleScn(
} catch (InterruptedException e) {
LOG.error("Send signal event error.", e);
errorHandler.setProducerThrowable(
new DebeziumException("Error processing binlog signal event", e));
new DebeziumException("Error processing redo log signal event", e));
}
// tell fetcher the binlog task finished
((OracleScanFetchTask.SnapshotBinlogSplitChangeEventSourceContext) context)
// tell fetcher the redo log task finished
((OracleScanFetchTask.SnapshotRedoLogSplitChangeEventSourceContext) context)
.finished();
}
}
Expand All @@ -179,7 +179,8 @@ public static RedoLogOffset getCurrentRedoLogOffset(Map<String, ?> offset) {
}

/**
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for binlog split task.
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for redo log split
* task.
*/
private class RedoLogSplitChangeEventSourceContext
implements ChangeEventSource.ChangeEventSourceContext {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A {@link DynamicTableSource} that describes how to create a Oracle binlog from a logical
* A {@link DynamicTableSource} that describes how to create a Oracle redo log from a logical
* description.
*/
public class OracleTableSource implements ScanTableSource, SupportsReadingMetadata {
Expand Down
Loading