Skip to content

Commit

Permalink
[cdc] Fix bug that computed columns work abnormally with case-insensi…
Browse files Browse the repository at this point in the history
…tive
  • Loading branch information
yuzelin committed Dec 12, 2023
1 parent 49a51d3 commit 57705e7
Show file tree
Hide file tree
Showing 25 changed files with 472 additions and 443 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.slf4j.LoggerFactory;

import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -162,12 +163,14 @@ public static String columnCaseConvertAndDuplicateCheck(
}

public static Schema buildPaimonSchema(
String tableName,
List<String> specifiedPartitionKeys,
List<String> specifiedPrimaryKeys,
List<ComputedColumn> computedColumns,
Map<String, String> tableConfig,
Schema sourceSchema,
CdcMetadataConverter[] metadataConverters,
boolean caseSensitive,
boolean requirePrimaryKeys) {
Schema.Builder builder = Schema.newBuilder();

Expand All @@ -176,40 +179,53 @@ public static Schema buildPaimonSchema(
builder.options(sourceSchema.options());

// fields
sourceSchema
.fields()
.forEach(
dataField ->
builder.column(
dataField.name(),
dataField.type(),
dataField.description()));
Set<String> existedFields = new HashSet<>();
Function<String, String> columnDuplicateErrMsg = columnDuplicateErrMsg(tableName);

for (DataField field : sourceSchema.fields()) {
String fieldName =
columnCaseConvertAndDuplicateCheck(
field.name(), existedFields, caseSensitive, columnDuplicateErrMsg);
builder.column(fieldName, field.type(), field.description());
}

for (ComputedColumn computedColumn : computedColumns) {
builder.column(computedColumn.columnName(), computedColumn.columnType());
String computedColumnName =
columnCaseConvertAndDuplicateCheck(
computedColumn.columnName(),
existedFields,
caseSensitive,
columnDuplicateErrMsg);
builder.column(computedColumnName, computedColumn.columnType());
}

for (CdcMetadataConverter metadataConverter : metadataConverters) {
builder.column(metadataConverter.columnName(), metadataConverter.dataType());
String metadataColumnName =
columnCaseConvertAndDuplicateCheck(
metadataConverter.columnName(),
existedFields,
caseSensitive,
columnDuplicateErrMsg);
builder.column(metadataColumnName, metadataConverter.dataType());
}

// primary keys
if (!specifiedPrimaryKeys.isEmpty()) {
Map<String, Integer> sourceColumns =
sourceSchema.fields().stream()
.collect(Collectors.toMap(DataField::name, entity -> 1));
Set<String> sourceColumns =
sourceSchema.fields().stream().map(DataField::name).collect(Collectors.toSet());
sourceColumns.addAll(
computedColumns.stream()
.map(ComputedColumn::columnName)
.collect(Collectors.toSet()));
for (String key : specifiedPrimaryKeys) {
if (!sourceColumns.containsKey(key)
&& computedColumns.stream().noneMatch(c -> c.columnName().equals(key))) {
throw new IllegalArgumentException(
"Specified primary key '"
+ key
+ "' does not exist in source tables or computed columns.");
}
checkArgument(
sourceColumns.contains(key),
"Specified primary '%s' does not exist in source tables or computed columns.",
key);
}
builder.primaryKey(specifiedPrimaryKeys);
builder.primaryKey(listCaseConvert(specifiedPrimaryKeys, caseSensitive));
} else if (!sourceSchema.primaryKeys().isEmpty()) {
builder.primaryKey(sourceSchema.primaryKeys());
builder.primaryKey(listCaseConvert(sourceSchema.primaryKeys(), caseSensitive));
} else if (requirePrimaryKeys) {
throw new IllegalArgumentException(
"Primary keys are not specified. "
Expand All @@ -219,7 +235,7 @@ public static Schema buildPaimonSchema(

// partition keys
if (!specifiedPartitionKeys.isEmpty()) {
builder.partitionKeys(specifiedPartitionKeys);
builder.partitionKeys(listCaseConvert(specifiedPartitionKeys, caseSensitive));
}

// comment
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataType;
import org.apache.paimon.utils.Preconditions;
import org.apache.paimon.utils.StringUtils;

import java.util.ArrayList;
import java.util.Arrays;
Expand All @@ -35,6 +36,12 @@ public class ComputedColumnUtils {

public static List<ComputedColumn> buildComputedColumns(
List<String> computedColumnArgs, List<DataField> physicFields) {
return buildComputedColumns(computedColumnArgs, physicFields, true);
}

/** The caseSensitive only affects check. We don't change field names at building phase. */
public static List<ComputedColumn> buildComputedColumns(
List<String> computedColumnArgs, List<DataField> physicFields, boolean caseSensitive) {
Map<String, DataType> typeMapping =
physicFields.stream()
.collect(
Expand Down Expand Up @@ -67,11 +74,13 @@ public static List<ComputedColumn> buildComputedColumns(
String fieldReference = args[0].trim();
String[] literals =
Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new);
String fieldReferenceCheckForm =
StringUtils.caseSensitiveConversion(fieldReference, caseSensitive);
checkArgument(
typeMapping.containsKey(fieldReference),
typeMapping.containsKey(fieldReferenceCheckForm),
String.format(
"Referenced field '%s' is not in given fields: %s.",
fieldReference, typeMapping.keySet()));
fieldReferenceCheckForm, typeMapping.keySet()));

computedColumns.add(
new ComputedColumn(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,12 +72,14 @@ protected Schema retrieveSchema() throws Exception {
@Override
protected Schema buildPaimonSchema(Schema retrievedSchema) {
return CdcActionCommonUtils.buildPaimonSchema(
table,
partitionKeys,
primaryKeys,
computedColumns,
tableConfig,
retrievedSchema,
metadataConverters,
caseSensitive,
false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -100,21 +100,21 @@ public SyncTableActionBase withComputedColumnArgs(List<String> computedColumnArg

protected Schema buildPaimonSchema(Schema retrievedSchema) {
return CdcActionCommonUtils.buildPaimonSchema(
table,
partitionKeys,
primaryKeys,
computedColumns,
tableConfig,
retrievedSchema,
metadataConverters,
caseSensitive,
true);
}

@Override
protected void validateCaseSensitivity() {
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Database", database);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Table", table);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Partition keys", partitionKeys);
AbstractCatalog.validateCaseInsensitive(caseSensitive, "Primary keys", primaryKeys);
}

@Override
Expand All @@ -124,7 +124,6 @@ protected void beforeBuildingSourceSink() throws Exception {
if (catalog.tableExists(identifier)) {
fileStoreTable = (FileStoreTable) catalog.getTable(identifier).copy(tableConfig);
try {
// TODO: test case insensitive with computed columns
Schema retrievedSchema = retrieveSchema();
computedColumns =
buildComputedColumns(computedColumnArgs, retrievedSchema.fields());
Expand All @@ -138,7 +137,10 @@ protected void beforeBuildingSourceSink() throws Exception {
+ "the Paimon table has defined all the argument columns used for computed columns.");
// schema evolution will add the computed columns
computedColumns =
buildComputedColumns(computedColumnArgs, fileStoreTable.schema().fields());
buildComputedColumns(
computedColumnArgs,
fileStoreTable.schema().fields(),
caseSensitive);
// check partition keys and primary keys in case that user specified them
checkConstraints();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,16 +43,12 @@

import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;

import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnCaseConvertAndDuplicateCheck;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnDuplicateErrMsg;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.listCaseConvert;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.mapKeyCaseConvert;
Expand Down Expand Up @@ -97,22 +93,9 @@ public Schema buildSchema(String record) {
return null;
}

LinkedHashMap<String, DataType> paimonFieldTypes = extractPaimonFieldTypes();
Schema.Builder builder = Schema.newBuilder();
Set<String> existedFields = new HashSet<>();
Function<String, String> columnDuplicateErrMsg = columnDuplicateErrMsg(getTableName());
for (Map.Entry<String, DataType> entry : paimonFieldTypes.entrySet()) {
builder.column(
columnCaseConvertAndDuplicateCheck(
entry.getKey(),
existedFields,
caseSensitive,
columnDuplicateErrMsg),
entry.getValue());
}

builder.primaryKey(listCaseConvert(extractPrimaryKeys(), caseSensitive));

extractPaimonFieldTypes().forEach(builder::column);
builder.primaryKey(extractPrimaryKeys());
return builder.build();
} catch (Exception e) {
logInvalidJsonString(record);
Expand Down Expand Up @@ -145,7 +128,7 @@ protected LinkedHashMap<String, DataType> fillDefaultStringTypes(JsonNode record
}

@Override
public void flatMap(String value, Collector<RichCdcMultiplexRecord> out) throws Exception {
public void flatMap(String value, Collector<RichCdcMultiplexRecord> out) {
try {
setRoot(value);
extractRecords().forEach(out::collect);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ public MongoDBSyncTableAction(

@Override
protected Schema retrieveSchema() {
return MongodbSchemaUtils.getMongodbSchema(cdcSourceConfig, caseSensitive);
return MongodbSchemaUtils.getMongodbSchema(cdcSourceConfig);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,11 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;

import static com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope.encodeValue;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnCaseConvertAndDuplicateCheck;
import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.columnDuplicateErrMsg;
import static org.apache.paimon.flink.action.cdc.mongodb.MongoDBActionUtils.FIELD_NAME;
import static org.apache.paimon.flink.action.cdc.mongodb.MongoDBActionUtils.START_MODE;

Expand Down Expand Up @@ -77,7 +72,7 @@ public class MongodbSchemaUtils {
* name, and optionally, the username and password for authentication. For the SPECIFIED mode,
* the field names should also be specified in the configuration.
*/
public static Schema getMongodbSchema(Configuration mongodbConfig, boolean caseSensitive) {
public static Schema getMongodbSchema(Configuration mongodbConfig) {
SchemaAcquisitionMode mode = getModeFromConfig(mongodbConfig);
String databaseName =
Objects.requireNonNull(
Expand All @@ -95,7 +90,7 @@ public static Schema getMongodbSchema(Configuration mongodbConfig, boolean caseS
mongodbConfig.get(FIELD_NAME), "Field names cannot be null")
.split(",");

return createMongodbSchema(collectionName, columnNames, caseSensitive);
return createMongodbSchema(columnNames);
case DYNAMIC:
String hosts =
Objects.requireNonNull(
Expand Down Expand Up @@ -126,8 +121,7 @@ public static Schema getMongodbSchema(Configuration mongodbConfig, boolean caseS
"No documents in collection to infer schema");
}

return createMongodbSchema(
collectionName, getColumnNames(firstDocument), caseSensitive);
return createMongodbSchema(getColumnNames(firstDocument));
} catch (Exception e) {
throw new RuntimeException(
"Failed to create schema from MongoDB collection", e);
Expand Down Expand Up @@ -166,21 +160,14 @@ private static List<String> getColumnNames(Document document) {
return document != null ? new ArrayList<>(document.keySet()) : Collections.emptyList();
}

private static Schema createMongodbSchema(
String collectionName, String[] columnNames, boolean caseSensitive) {
return createMongodbSchema(collectionName, Arrays.asList(columnNames), caseSensitive);
private static Schema createMongodbSchema(String[] columnNames) {
return createMongodbSchema(Arrays.asList(columnNames));
}

private static Schema createMongodbSchema(
String collectionName, List<String> columnNames, boolean caseSensitive) {
private static Schema createMongodbSchema(List<String> columnNames) {
Schema.Builder builder = Schema.newBuilder();
Set<String> existedFields = new HashSet<>();
Function<String, String> columnDuplicateErrMsg = columnDuplicateErrMsg(collectionName);
for (String column : columnNames) {
builder.column(
columnCaseConvertAndDuplicateCheck(
column, existedFields, caseSensitive, columnDuplicateErrMsg),
DataTypes.STRING());
builder.column(column, DataTypes.STRING());
}

builder.primaryKey(ID_FIELD);
Expand Down
Loading

0 comments on commit 57705e7

Please sign in to comment.