JingsongLi commented on code in PR #931: URL: https://github.com/apache/incubator-paimon/pull/931#discussion_r1169414071
########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunctionBase.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.paimon.flink.sink.cdc; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; + +/** Base class of {@link ProcessFunction} for CDC parsing. */ +public abstract class CdcParsingProcessFunctionBase<T, O> extends ProcessFunction<T, O> { Review Comment: No need to create an abstract class... ########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java: ########## @@ -154,14 +162,23 @@ public void build(StreamExecutionEnvironment env) throws Exception { env.fromSource( source, WatermarkStrategy.noWatermarks(), "MySQL Source")) .withParserFactory(parserFactory) - .withTable(table); + .withTable(table) + .withCaseSensitive(catalog.caseSensitive()); String sinkParallelism = paimonConfig.get(FlinkConnectorOptions.SINK_PARALLELISM.key()); if (sinkParallelism != null) { sinkBuilder.withParallelism(Integer.parseInt(sinkParallelism)); } sinkBuilder.build(); } + private void toCaseInsensitiveForm() { + database = database.toLowerCase(); Review Comment: Only validate here is OK ########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecord.java: ########## @@ -131,6 +132,20 @@ public Optional<GenericRow> toGenericRow(List<DataField> dataFields) { return Optional.of(genericRow); } + public CdcRecord toCaseInsensitiveForm() { Review Comment: It is better to just lower case in creation. ########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java: ########## @@ -124,7 +124,15 @@ public void build(StreamExecutionEnvironment env) throws Exception { Catalog catalog = CatalogFactory.createCatalog( CatalogContext.create( - new Options().set(CatalogOptions.WAREHOUSE, warehouse))); + new Options(paimonConfig) + .set(CatalogOptions.WAREHOUSE, warehouse))); + MySqlActionUtils.removeTableDefaultOptions(paimonConfig); Review Comment: Maybe we can create a table options and a catalog options. ########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSchema.java: ########## @@ -31,8 +31,8 @@ /** Utility class to load MySQL table schema with JDBC. */ public class MySqlSchema { - private final String databaseName; - private final String tableName; + private String databaseName; + private String tableName; private final Map<String, DataType> fields; Review Comment: It should be a `LinkedHashMap` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@paimon.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org