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 fix 1875 #2215

Merged
merged 5 commits into from
Jun 15, 2023
Merged
Show file tree
Hide file tree
Changes from 3 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,6 +110,8 @@ public Iterator<SourceRecords> pollSplitRecords() throws InterruptedException {
for (DataChangeEvent event : batch) {
if (shouldEmit(event.getRecord())) {
sourceRecords.add(event.getRecord());
} else {
LOG.debug("{} data change event should not emit", event);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ public CommonConnectorConfig getDbzConnectorConfig() {
}

public SchemaNameAdjuster getSchemaNameAdjuster() {
return null;
return schemaNameAdjuster;
}

public abstract RelationalDatabaseSchema getDatabaseSchema();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,8 @@ public Collection<SnapshotSplit> generateSplits(TableId tableId) {
long start = System.currentTimeMillis();

Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
Column splitColumn = getSplitColumn(table, sourceConfig);
Column splitColumn =
ChunkUtils.getChunkKeyColumn(table, sourceConfig.getChunkKeyColumn());
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
Expand Down Expand Up @@ -377,8 +378,4 @@ private static void maySleep(int count, TableId tableId) {
LOG.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId);
}
}

public static Column getSplitColumn(Table table, JdbcSourceConfig sourceConfig) {
return ChunkUtils.getChunkKeyColumn(table, sourceConfig.getChunkKeyColumn());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,11 @@
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext;
import com.ververica.cdc.connectors.base.utils.SourceRecordUtils;
import com.ververica.cdc.connectors.oracle.source.config.OracleSourceConfig;
import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset;
import com.ververica.cdc.connectors.oracle.source.utils.OracleUtils;
import com.ververica.cdc.connectors.oracle.util.ChunkUtils;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.oracle.OracleChangeEventSourceMetricsFactory;
import io.debezium.connector.oracle.OracleConnection;
Expand All @@ -52,14 +54,19 @@
import io.debezium.schema.DataCollectionId;
import io.debezium.schema.TopicSelector;
import io.debezium.util.Collect;
import oracle.sql.ROWID;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.source.SourceRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.sql.SQLException;
import java.time.Instant;
import java.util.Map;

import static com.ververica.cdc.connectors.oracle.util.ChunkUtils.getChunkKeyColumn;

/** The context for fetch task that fetching data of snapshot split from Oracle data source. */
public class OracleSourceFetchTaskContext extends JdbcSourceFetchTaskContext {

Expand Down Expand Up @@ -186,7 +193,43 @@ public OracleDatabaseSchema getDatabaseSchema() {

@Override
public RowType getSplitType(Table table) {
return OracleUtils.getSplitType(table);
OracleSourceConfig oracleSourceConfig = getSourceConfig();
return ChunkUtils.getSplitType(
getChunkKeyColumn(table, oracleSourceConfig.getChunkKeyColumn()));
}

@Override
public boolean isDataChangeRecord(SourceRecord record) {
return SourceRecordUtils.isDataChangeRecord(record);
}

@Override
public boolean isRecordBetween(SourceRecord record, Object[] splitStart, Object[] splitEnd) {
RowType splitKeyType =
getSplitType(getDatabaseSchema().tableFor(SourceRecordUtils.getTableId(record)));

// RowId is chunk key column by default, compare RowId
if (splitKeyType.getFieldNames().contains(ROWID.class.getSimpleName())) {
ConnectHeaders headers = (ConnectHeaders) record.headers();
ROWID rowId = null;
try {
rowId = new ROWID(headers.iterator().next().value().toString());
} catch (SQLException e) {
LOG.error("{} can not convert to RowId", record);
}
Object[] rowIds = new ROWID[] {rowId};
return SourceRecordUtils.splitKeyRangeContains(rowIds, splitStart, splitEnd);
} else {
// config chunk key column compare
Object[] key =
SourceRecordUtils.getSplitKey(splitKeyType, record, getSchemaNameAdjuster());
return SourceRecordUtils.splitKeyRangeContains(key, splitStart, splitEnd);
}
}

@Override
public TableId getTableId(SourceRecord record) {
return SourceRecordUtils.getTableId(record);
}

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

package com.ververica.cdc.connectors.oracle.source.utils;

import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.types.logical.RowType;

import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset;
Expand All @@ -28,8 +27,6 @@
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.StreamingAdapter;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.schema.TopicSelector;
import io.debezium.util.SchemaNameAdjuster;
Expand All @@ -40,14 +37,11 @@
import java.sql.SQLException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;

import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
import static org.apache.flink.table.api.DataTypes.FIELD;
import static org.apache.flink.table.api.DataTypes.ROW;

/** Utils to prepare Oracle SQL statement. */
public class OracleUtils {
Expand Down Expand Up @@ -245,20 +239,6 @@ public static PreparedStatement readTableSplitDataStatement(
}
}

public static RowType getSplitType(Table table) {
List<Column> primaryKeys = table.primaryKeyColumns();
if (primaryKeys.isEmpty()) {
throw new ValidationException(
String.format(
"Incremental snapshot for tables requires primary key,"
+ " but table %s doesn't have primary key.",
table.id()));
}

// use first field in primary key as the split key
return getSplitType(primaryKeys.get(0));
}

/** Creates a new {@link OracleDatabaseSchema} to monitor the latest oracle database schemas. */
public static OracleDatabaseSchema createOracleDatabaseSchema(
OracleConnectorConfig dbzOracleConfig) {
Expand Down Expand Up @@ -313,26 +293,6 @@ public static RedoLogOffset getRedoLogPosition(Map<String, ?> offset) {
return new RedoLogOffset(offsetStrMap);
}

public static RowType getSplitType(Column splitColumn) {
return (RowType)
ROW(FIELD(splitColumn.name(), OracleTypeUtils.fromDbzColumn(splitColumn)))
.getLogicalType();
}

public static Column getSplitColumn(Table table) {
List<Column> primaryKeys = table.primaryKeyColumns();
if (primaryKeys.isEmpty()) {
throw new ValidationException(
String.format(
"Incremental snapshot for tables requires primary key,"
+ " but table %s doesn't have primary key.",
table.id()));
}

// use first field in primary key as the split key
return primaryKeys.get(0);
}

public static String quote(String dbOrTableName) {
return "\"" + dbOrTableName + "\"";
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@
package com.ververica.cdc.connectors.oracle.util;

import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.types.logical.RowType;

import com.ververica.cdc.connectors.oracle.source.utils.OracleTypeUtils;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import oracle.sql.ROWID;
Expand All @@ -29,11 +31,20 @@
import java.util.Optional;
import java.util.stream.Collectors;

import static org.apache.flink.table.api.DataTypes.FIELD;
import static org.apache.flink.table.api.DataTypes.ROW;

/** Utilities to split chunks of table. */
public class ChunkUtils {

private ChunkUtils() {}

public static RowType getSplitType(Column splitColumn) {
return (RowType)
ROW(FIELD(splitColumn.name(), OracleTypeUtils.fromDbzColumn(splitColumn)))
.getLogicalType();
}

public static Column getChunkKeyColumn(Table table, @Nullable String chunkKeyColumn) {
List<Column> primaryKeys = table.primaryKeyColumns();

Expand Down
Loading