Skip to content

Commit

Permalink
Merge remote-tracking branch 'ops/2.4.8-release' into 2.4.8-release
Browse files Browse the repository at this point in the history
  • Loading branch information
ic4y committed Jul 19, 2023
2 parents 6eb3428 + 8d32d5f commit bc4b6b9
Show file tree
Hide file tree
Showing 17 changed files with 255 additions and 146 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.seatunnel.connectors.cdc.base.dialect;

import org.apache.seatunnel.api.table.catalog.ConstraintKey;
import org.apache.seatunnel.api.table.catalog.PrimaryKey;
import org.apache.seatunnel.common.utils.SeaTunnelException;
import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig;
import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionFactory;
Expand All @@ -25,11 +27,23 @@
import org.apache.seatunnel.connectors.cdc.base.source.reader.external.JdbcSourceFetchTaskContext;
import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase;

import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.tuple.Pair;

import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.TableId;
import io.debezium.relational.history.TableChanges;

import java.sql.DatabaseMetaData;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;

public interface JdbcDataSourceDialect extends DataSourceDialect<JdbcSourceConfig> {

Expand Down Expand Up @@ -68,4 +82,90 @@ default JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) {
@Override
JdbcSourceFetchTaskContext createFetchTaskContext(
SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig);

default Optional<PrimaryKey> getPrimaryKey(JdbcConnection jdbcConnection, TableId tableId)
throws SQLException {

DatabaseMetaData metaData = jdbcConnection.connection().getMetaData();

// According to the Javadoc of java.sql.DatabaseMetaData#getPrimaryKeys,
// the returned primary key columns are ordered by COLUMN_NAME, not by KEY_SEQ.
// We need to sort them based on the KEY_SEQ value.
ResultSet rs =
metaData.getPrimaryKeys(tableId.catalog(), tableId.schema(), tableId.table());

// seq -> column name
List<Pair<Integer, String>> primaryKeyColumns = new ArrayList<>();
String pkName = null;
while (rs.next()) {
// all the PK_NAME should be the same
pkName = rs.getString("PK_NAME");
String columnName = rs.getString("COLUMN_NAME");
int keySeq = rs.getInt("KEY_SEQ");
// KEY_SEQ is 1-based index
primaryKeyColumns.add(Pair.of(keySeq, columnName));
}
// initialize size
List<String> pkFields =
primaryKeyColumns.stream()
.sorted(Comparator.comparingInt(Pair::getKey))
.map(Pair::getValue)
.collect(Collectors.toList());
if (CollectionUtils.isEmpty(pkFields)) {
return Optional.empty();
}
return Optional.of(PrimaryKey.of(pkName, pkFields));
}

default List<ConstraintKey> getUniqueKeys(JdbcConnection jdbcConnection, TableId tableId)
throws SQLException {
return getConstraintKeys(jdbcConnection, tableId).stream()
.filter(
constraintKey ->
constraintKey.getConstraintType()
== ConstraintKey.ConstraintType.UNIQUE_KEY)
.collect(Collectors.toList());
}

default List<ConstraintKey> getConstraintKeys(JdbcConnection jdbcConnection, TableId tableId)
throws SQLException {
DatabaseMetaData metaData = jdbcConnection.connection().getMetaData();

ResultSet resultSet =
metaData.getIndexInfo(
tableId.catalog(), tableId.schema(), tableId.table(), false, false);
// index name -> index
Map<String, ConstraintKey> constraintKeyMap = new HashMap<>();
while (resultSet.next()) {
String columnName = resultSet.getString("COLUMN_NAME");
if (columnName == null) {
continue;
}

String indexName = resultSet.getString("INDEX_NAME");
boolean noUnique = resultSet.getBoolean("NON_UNIQUE");

ConstraintKey constraintKey =
constraintKeyMap.computeIfAbsent(
indexName,
s -> {
ConstraintKey.ConstraintType constraintType =
ConstraintKey.ConstraintType.KEY;
if (!noUnique) {
constraintType = ConstraintKey.ConstraintType.UNIQUE_KEY;
}
return ConstraintKey.of(
constraintType, indexName, new ArrayList<>());
});

ConstraintKey.ColumnSortType sortType =
"A".equals(resultSet.getString("ASC_OR_DESC"))
? ConstraintKey.ColumnSortType.ASC
: ConstraintKey.ColumnSortType.DESC;
ConstraintKey.ConstraintKeyColumn constraintKeyColumn =
new ConstraintKey.ConstraintKeyColumn(columnName, sortType);
constraintKey.getColumnNames().add(constraintKeyColumn);
}
return new ArrayList<>(constraintKeyMap.values());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,22 @@

package org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter;

import org.apache.seatunnel.api.table.catalog.ConstraintKey;
import org.apache.seatunnel.api.table.catalog.PrimaryKey;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect;
import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit;

import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;

import java.sql.SQLException;
import java.util.Collection;
import java.util.List;
import java.util.Optional;

/** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */
public interface JdbcSourceChunkSplitter extends ChunkSplitter {
Expand Down Expand Up @@ -161,4 +167,42 @@ default SeaTunnelRowType getSplitType(Column splitColumn) {
new String[] {splitColumn.name()},
new SeaTunnelDataType[] {fromDbzColumn(splitColumn)});
}

default Column getSplitColumn(
JdbcConnection jdbc, JdbcDataSourceDialect dialect, TableId tableId)
throws SQLException {
Optional<PrimaryKey> primaryKey = dialect.getPrimaryKey(jdbc, tableId);
if (primaryKey.isPresent()) {
List<String> pkColumns = primaryKey.get().getColumnNames();

Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
for (String pkColumn : pkColumns) {
Column column = table.columnWithName(pkColumn);
if (isEvenlySplitColumn(column)) {
return column;
}
}
}

List<ConstraintKey> uniqueKeys = dialect.getUniqueKeys(jdbc, tableId);
if (!uniqueKeys.isEmpty()) {
Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
for (ConstraintKey uniqueKey : uniqueKeys) {
List<ConstraintKey.ConstraintKeyColumn> uniqueKeyColumns =
uniqueKey.getColumnNames();
for (ConstraintKey.ConstraintKeyColumn uniqueKeyColumn : uniqueKeyColumns) {
Column column = table.columnWithName(uniqueKeyColumn.getColumnName());
if (isEvenlySplitColumn(column)) {
return column;
}
}
}
}

throw new UnsupportedOperationException(
String.format(
"Incremental snapshot for tables requires primary key/unique key,"
+ " but table %s doesn't have primary key.",
tableId));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId ta
if (damengSchema == null) {
synchronized (this) {
if (damengSchema == null) {
damengSchema = new DamengSchema();
damengSchema = new DamengSchema(sourceConfig.getDbzConnectorConfig());
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@

import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import lombok.AllArgsConstructor;
import lombok.extern.slf4j.Slf4j;
Expand Down Expand Up @@ -59,8 +58,7 @@ public Collection<SnapshotSplit> generateSplits(TableId tableId) {
log.info("Start splitting table {} into chunks...", tableId);
long start = System.currentTimeMillis();

Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
Column splitColumn = getSplitColumn(table);
Column splitColumn = getSplitColumn(jdbc, dialect, tableId);
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
Expand Down Expand Up @@ -150,23 +148,6 @@ public SeaTunnelDataType<?> fromDbzColumn(Column splitColumn) {
return DamengTypeConverter.convert(splitColumn);
}

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

Column splitColumn = primaryKeys.get(0);
if (primaryKeys.size() > 1) {
log.warn("Using first field[{}] in primary key as the split key", splitColumn);
}
return splitColumn;
}

@SuppressWarnings("MagicNumber")
private double calculateDistributionFactor(
TableId tableId, Object min, Object max, long approximateRowCnt) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,43 +17,68 @@

package org.apache.seatunnel.connectors.cdc.dameng.utils;

import org.apache.seatunnel.common.exception.CommonErrorCode;
import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException;
import org.apache.seatunnel.common.utils.SeaTunnelException;

import io.debezium.connector.dameng.DamengConnection;
import io.debezium.connector.dameng.DamengConnectorConfig;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.relational.history.TableChanges;

import java.sql.SQLException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

public class DamengSchema {
private final Map<TableId, TableChanges.TableChange> schemasByTableId = new HashMap<>();
private final DamengConnectorConfig connectorConfig;
private final Map<TableId, TableChanges.TableChange> schemasByTableId;

public DamengSchema(DamengConnectorConfig connectorConfig) {
this.connectorConfig = connectorConfig;
this.schemasByTableId = new HashMap<>();
}

public TableChanges.TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) {
// read schema from cache first
TableChanges.TableChange schema = schemasByTableId.get(tableId);
if (schema == null) {
try {
List<Column> columns = DamengConncetionUtils.queryColumns(jdbc, tableId);
List<String> primaryKeyNames =
DamengConncetionUtils.queryPrimaryKeyNames(jdbc, tableId);
Table table =
Table.editor()
.tableId(tableId)
.setColumns(columns)
.setPrimaryKeyNames(primaryKeyNames)
.create();
schema = new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table);
schemasByTableId.put(tableId, schema);
} catch (SQLException e) {
throw new SeaTunnelRuntimeException(CommonErrorCode.SQL_OPERATION_FAILED, e);
}
schema = readTableSchema(jdbc, tableId);
schemasByTableId.put(tableId, schema);
}
return schema;
}

private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) {
DamengConnection damengConnection = (DamengConnection) jdbc;
// read schema from cache first
final Map<TableId, TableChanges.TableChange> tableChangeMap = new HashMap<>();
Tables tables = new Tables();

try {
damengConnection.readSchema(
tables,
tableId.catalog(),
tableId.schema(),
connectorConfig.getTableFilters().dataCollectionFilter(),
null,
false);

Table table = tables.forTable(tableId);
TableChanges.TableChange tableChange =
new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table);
tableChangeMap.put(tableId, tableChange);
} catch (SQLException e) {
throw new SeaTunnelException(
String.format("Failed to read schema for table %s ", tableId), e);
}

if (!tableChangeMap.containsKey(tableId)) {
throw new SeaTunnelException(
String.format("Can't obtain schema for table %s ", tableId));
}

return tableChangeMap.get(tableId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@

import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;

import java.math.BigDecimal;
Expand Down Expand Up @@ -67,8 +66,7 @@ public Collection<SnapshotSplit> generateSplits(TableId tableId) {
LOG.info("Start splitting table {} into chunks...", tableId);
long start = System.currentTimeMillis();

Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
Column splitColumn = getSplitColumn(table);
Column splitColumn = getSplitColumn(jdbc, dialect, tableId);
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
Expand Down Expand Up @@ -393,18 +391,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) {
List<Column> primaryKeys = table.primaryKeyColumns();
if (primaryKeys.isEmpty()) {
throw new UnsupportedOperationException(
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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ public List<TableId> discoverDataCollections(JdbcSourceConfig sourceConfig) {
@Override
public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) {
if (oracleSchema == null) {
oracleSchema = new OracleSchema();
oracleSchema = new OracleSchema(sourceConfig.getDbzConnectorConfig());
}
return oracleSchema.getTableSchema(jdbc, tableId);
}
Expand Down
Loading

0 comments on commit bc4b6b9

Please sign in to comment.