Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-07-02 Thread via GitHub


loserwang1024 commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1657064426


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -132,16 +172,237 @@ default boolean isEvenlySplitColumn(Column splitColumn) {
  * @param splitColumn dbz split column.
  * @return flink data type
  */
-DataType fromDbzColumn(Column splitColumn);
+protected abstract DataType fromDbzColumn(Column splitColumn);
+
+/** Returns the distribution factor of the given table. */
+protected double calculateDistributionFactor(
+TableId tableId, Object min, Object max, long approximateRowCnt) {
+
+if (!min.getClass().equals(max.getClass())) {
+throw new IllegalStateException(
+String.format(
+"Unsupported operation type, the MIN value type %s 
is different with MAX value type %s.",
+min.getClass().getSimpleName(), 
max.getClass().getSimpleName()));
+}
+if (approximateRowCnt == 0) {
+return Double.MAX_VALUE;
+}
+BigDecimal difference = ObjectUtils.minus(max, min);
+// factor = (max - min + 1) / rowCount
+final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1));
+double distributionFactor =
+subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, 
ROUND_CEILING).doubleValue();
+LOG.info(
+"The distribution factor of table {} is {} according to the 
min split key {}, max split key {} and approximate row count {}",
+tableId,
+distributionFactor,
+min,
+max,
+approximateRowCnt);
+return distributionFactor;
+}
+
+/**
+ * Get the column which is seen as chunk key.
+ *
+ * @param table table identity.
+ * @param chunkKeyColumn column name which is seen as chunk key, if 
chunkKeyColumn is null, use
+ * primary key instead. @Column the column which is seen as chunk key.
+ */
+protected Column getSplitColumn(Table table, @Nullable String 
chunkKeyColumn) {
+return JdbcChunkUtils.getSplitColumn(table, chunkKeyColumn);
+}
+
+/** ChunkEnd less than or equal to max. */
+protected boolean isChunkEndLeMax(Object chunkEnd, Object max) {
+return ObjectUtils.compare(chunkEnd, max) <= 0;
+}
+
+/** ChunkEnd greater than or equal to max. */
+protected boolean isChunkEndGeMax(Object chunkEnd, Object max) {
+return ObjectUtils.compare(chunkEnd, max) >= 0;
+}
 
 /**
  * convert dbz column to Flink row type.
  *
  * @param splitColumn split column.
  * @return flink row type.
  */
-default RowType getSplitType(Column splitColumn) {
+private RowType getSplitType(Column splitColumn) {
 return (RowType)
 ROW(FIELD(splitColumn.name(), 
fromDbzColumn(splitColumn))).getLogicalType();
 }
+
+/**
+ * We can use evenly-sized chunks or unevenly-sized chunks when split 
table into chunks, using
+ * evenly-sized chunks which is much efficient, using unevenly-sized 
chunks which will request
+ * many queries and is not efficient.
+ */
+private List splitTableIntoChunks(
+JdbcConnection jdbc, TableId tableId, Column splitColumn) throws 
SQLException {
+final String splitColumnName = splitColumn.name();
+final Object[] minMax = JdbcChunkUtils.queryMinMax(jdbc, tableId, 
splitColumnName);
+final Object min = minMax[0];
+final Object max = minMax[1];
+if (min == null || max == null || min.equals(max)) {
+// empty table, or only one row, return full table scan as a chunk
+return Collections.singletonList(ChunkRange.all());
+}
+
+final int chunkSize = sourceConfig.getSplitSize();
+final double distributionFactorUpper = 
sourceConfig.getDistributionFactorUpper();
+final double distributionFactorLower = 
sourceConfig.getDistributionFactorLower();
+
+if (isEvenlySplitColumn(splitColumn)) {
+long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId);
+double distributionFactor =
+calculateDistributionFactor(tableId, min, max, 
approximateRowCnt);
+
+boolean dataIsEvenlyDistributed =
+doubleCompare(distributionFactor, distributionFactorLower) 
>= 0
+&& doubleCompare(distributionFactor, 
distributionFactorUpper) <= 0;
+
+if (dataIsEvenlyDistributed) {
+// the minimum dynamic chunk size is at least 1
+final int dynamicChunkSize = Math.max((int) 
(distributionFactor * chunkSize), 1);
+return splitEvenlySizedChunks(
+  

Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-14 Thread via GitHub


loserwang1024 commented on PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#issuecomment-2111625519

   > Could @loserwang1024 please take a look?
   I'd like to do it. But this PR do nothing with mysql cdc.
   
   


-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-14 Thread via GitHub


yuxiqian commented on PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#issuecomment-2111557242

   Seems MySQL CI is taking significantly more time (90min vs. 30min) and fails 
eventually. Could @loserwang1024 please take a look?
   
   Failed CI job link: 
https://github.com/apache/flink-cdc/actions/runs/9073569533/job/24931065832


-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-14 Thread via GitHub


loserwang1024 commented on PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#issuecomment-2111475946

   @PatrickRen , @ruanhang1993 , @leonardBang , CC


-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-14 Thread via GitHub


yuxiqian commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599426381


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   Got it, make sense to me.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-14 Thread via GitHub


yuxiqian commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599426381


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   Got it, makes sense to me.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


loserwang1024 commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599417206


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtil.java:
##
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.cdc.connectors.base.source.utils;
+
+import org.apache.flink.table.api.ValidationException;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
+
+/** Utilities to split chunks of table. */
+public class JdbcChunkUtil {
+
+/**
+ * Query the maximum and minimum value of the column in the table. e.g. 
query string 
+ * SELECT MIN(%s) FROM %s WHERE %s > ?
+ *
+ * @param jdbc JDBC connection.
+ * @param tableId table identity.
+ * @param columnName column name.
+ * @return maximum and minimum value.
+ */
+public static Object[] queryMinMax(JdbcConnection jdbc, String tableId, 
String columnName)
+throws SQLException {
+final String minMaxQuery =
+String.format("SELECT MIN(%s), MAX(%s) FROM %s", columnName, 
columnName, tableId);

Review Comment:
   Thanks, it looks better!



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


loserwang1024 commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599410767


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtil.java:
##
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.cdc.connectors.base.source.utils;
+
+import org.apache.flink.table.api.ValidationException;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
+
+/** Utilities to split chunks of table. */
+public class JdbcChunkUtil {

Review Comment:
   done it.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


loserwang1024 commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599410084


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   1. Interface  has no constructor. Most of the code needs sourceConfig and 
dialect.
   2. JdbcSourceChunkSplitter now is as a  template class.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


whhe commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599378513


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtil.java:
##
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.cdc.connectors.base.source.utils;
+
+import org.apache.flink.table.api.ValidationException;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
+
+/** Utilities to split chunks of table. */
+public class JdbcChunkUtil {
+
+/**
+ * Query the maximum and minimum value of the column in the table. e.g. 
query string 
+ * SELECT MIN(%s) FROM %s WHERE %s > ?
+ *
+ * @param jdbc JDBC connection.
+ * @param tableId table identity.
+ * @param columnName column name.
+ * @return maximum and minimum value.
+ */
+public static Object[] queryMinMax(JdbcConnection jdbc, String tableId, 
String columnName)
+throws SQLException {
+final String minMaxQuery =
+String.format("SELECT MIN(%s), MAX(%s) FROM %s", columnName, 
columnName, tableId);

Review Comment:
   How about using `jdbc.quotedTableIdString` and `jdbc.quotedColumnIdString` 
to quote tableId and columnName.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


yuxiqian commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599352772


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   Is there a reason to change `Splitter` interface to an abstract class, 
instead of providing a `default` implementation for `generateSplits`?



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


yuxiqian commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599352772


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   Is there a reason to change `Splitter` interface into an abstract class, 
instead of providing a `default` implementation for `generateSplits`?



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-35344][cdc-base] Move same code from multiple subclasses to JdbcSourceChunkSplitter [flink-cdc]

2024-05-13 Thread via GitHub


yuxiqian commented on code in PR #3319:
URL: https://github.com/apache/flink-cdc/pull/3319#discussion_r1599352772


##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java:
##
@@ -18,104 +18,144 @@
 package org.apache.flink.cdc.connectors.base.source.assigner.splitter;
 
 import org.apache.flink.cdc.common.annotation.Experimental;
+import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
+import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect;
 import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
+import org.apache.flink.cdc.connectors.base.source.utils.JdbcChunkUtil;
+import org.apache.flink.cdc.connectors.base.utils.ObjectUtils;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.relational.Column;
+import io.debezium.relational.Table;
 import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import static java.math.BigDecimal.ROUND_CEILING;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
 import static org.apache.flink.table.api.DataTypes.FIELD;
 import static org.apache.flink.table.api.DataTypes.ROW;
 
 /** The {@code ChunkSplitter} used to split table into a set of chunks for 
JDBC data source. */
 @Experimental
-public interface JdbcSourceChunkSplitter extends ChunkSplitter {

Review Comment:
   Is there any reason to change `Splitter` interface into an abstract class, 
instead of providing a `default` implementation for `generateSplits`?



##
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtil.java:
##
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.cdc.connectors.base.source.utils;
+
+import org.apache.flink.table.api.ValidationException;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
+
+/** Utilities to split chunks of table. */
+public class JdbcChunkUtil {

Review Comment:
   Seems `Utils` are used more than `Util` in CDC repo.



-- 
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...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org