ruanhang1993 commented on code in PR #180:
URL:
https://github.com/apache/flink-connector-jdbc/pull/180#discussion_r3212600720
##########
flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/PreparedSplitterNumericParametersTest.java:
##########
@@ -0,0 +1,116 @@
+package
org.apache.flink.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.Serializable;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class PreparedSplitterNumericParametersTest {
+
+ @Test
+ void testBatchSizeDivisible() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(-5,
9).withBatchSize(3).getParameterValues();
+
+ long[][] expected = {
+ new long[] {-5, -3},
+ new long[] {-2, 0},
+ new long[] {1, 3},
+ new long[] {4, 6},
+ new long[] {7, 9}
+ };
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchSizeNotDivisible() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(-5,
11).withBatchSize(4).getParameterValues();
+
+ long[][] expected = {
+ new long[] {-5, -2},
+ new long[] {-1, 2},
+ new long[] {3, 5},
+ new long[] {6, 8},
+ new long[] {9, 11}
+ };
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchSizeTooLarge() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(0,
2).withBatchSize(5).getParameterValues();
+
+ long[][] expected = {new long[] {0, 2}};
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchNumDivisible() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(-5,
9).withBatchNum(5).getParameterValues();
+
+ long[][] expected = {
+ new long[] {-5, -3},
+ new long[] {-2, 0},
+ new long[] {1, 3},
+ new long[] {4, 6},
+ new long[] {7, 9}
+ };
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchNumNotDivisible() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(-5,
11).withBatchNum(5).getParameterValues();
+
+ long[][] expected = {
+ new long[] {-5, -2},
+ new long[] {-1, 2},
+ new long[] {3, 5},
+ new long[] {6, 8},
+ new long[] {9, 11}
+ };
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchNumTooLarge() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(0,
2).withBatchNum(5).getParameterValues();
+
+ long[][] expected = {
+ new long[] {0, 0},
+ new long[] {1, 1},
+ new long[] {2, 2}
+ };
+ check(expected, parameters);
+ }
+
+ @Test
+ void testBatchMaxMinTooLarge() {
+ Serializable[][] parameters =
+ new PreparedSplitterNumericParameters(2260418954055131340L,
3875220057236942850L)
+ .withBatchNum(3)
+ .getParameterValues();
+
+ long[][] expected = {
+ new long[] {2260418954055131340L, 2798685988449068491L},
+ new long[] {2798685988449068492L, 3336953022843005643L},
+ new long[] {3336953022843005644L, 3875220057236942795L}
Review Comment:
The last expected element of the array should be `new long[]
{3336953022843005644L, 3875220057236942850L}` instead of `new long[]
{3336953022843005644L, 3875220057236942795L}`, right?
##########
flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/PreparedSplitterNumericParametersTest.java:
##########
@@ -0,0 +1,116 @@
+package
org.apache.flink.connector.jdbc.core.datastream.source.enumerator.splitter;
Review Comment:
Add apache licenses.
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/PreparedSplitterNumericParameters.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * This query parameters generator is a helper class to parameterize from/to
queries on a numeric
+ * column. The generated array of from/to values will be equally sized to
batchSize (apart from the
+ * last one), ranging from minVal up to maxVal.
+ */
+@Internal
+public class PreparedSplitterNumericParameters implements Serializable {
+
+ private final long minVal;
+ private final long maxVal;
+ private long batchSize;
+ private int batchNum;
+
+ public PreparedSplitterNumericParameters(long minVal, long maxVal) {
+ this.minVal = minVal;
+ this.maxVal = maxVal;
+ this.batchNum = 0;
+ this.batchSize = 0;
+ }
+
+ public PreparedSplitterNumericParameters withBatchSize(long batchSize) {
+ Preconditions.checkArgument(batchSize > 0, "Batch size must be
positive");
+
+ long maxElemCount = (maxVal - minVal) + 1;
+ if (batchSize > maxElemCount) {
+ batchSize = maxElemCount;
+ }
+ this.batchSize = batchSize;
+ this.batchNum = new Double(Math.ceil((double) maxElemCount /
batchSize)).intValue();
+ return this;
+ }
+
+ public PreparedSplitterNumericParameters withBatchNum(int batchNum) {
+ Preconditions.checkArgument(batchNum > 0, "Batch number must be
positive");
+
+ long maxElemCount = (maxVal - minVal) + 1;
+ if (batchNum > maxElemCount) {
+ batchNum = (int) maxElemCount;
+ }
+ this.batchNum = batchNum;
+ this.batchSize = new Double(Math.ceil((double) maxElemCount /
batchNum)).longValue();
Review Comment:
withBatchNum() uses Math.ceil((double) maxElemCount / batchNum) which loses
precision for large long values.
```java
this.batchSize = (maxElemCount + batchNum - 1) / batchNum;
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/PreparedSplitterEnumerator.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class PreparedSplitterEnumerator extends SqlSplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(PreparedSplitterEnumerator.class);
Review Comment:
```suggestion
private static final Logger LOG =
LoggerFactory.getLogger(PreparedSplitterEnumerator.class);
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/JdbcSourceBuilder.java:
##########
@@ -289,36 +302,52 @@ public JdbcSource<OUT> build() {
JdbcSourceOptions.READER_FETCH_BATCH_SIZE,
splitReaderFetchBatchSize);
this.configuration.set(JdbcSourceOptions.AUTO_COMMIT, autoCommit);
- Preconditions.checkState(
- !StringUtils.isNullOrWhitespaceOnly(sql), "'sql' mustn't be
null or empty.");
Preconditions.checkNotNull(resultExtractor, "'resultExtractor' mustn't
be null.");
Preconditions.checkNotNull(typeInformation, "'typeInformation' mustn't
be null.");
- if (Objects.nonNull(continuousUnBoundingSettings)) {
- Preconditions.checkArgument(
- Objects.nonNull(jdbcParameterValuesProvider)
- && jdbcParameterValuesProvider
- instanceof
JdbcSlideTimingParameterProvider,
- INVALID_SLIDE_TIMING_CONTINUOUS_HINT);
- }
-
- if (Objects.nonNull(jdbcParameterValuesProvider)
- && jdbcParameterValuesProvider instanceof
JdbcSlideTimingParameterProvider) {
- Preconditions.checkArgument(
- Objects.nonNull(continuousUnBoundingSettings),
- INVALID_CONTINUOUS_SLIDE_TIMING_HINT);
+ if (this.splitterEnumerator == null) {
Review Comment:
When a user calls both setSplitter() and the deprecated setSql() /
setJdbcParameterValuesProvider(), the build() method silently ignores the
legacy configuration. This can lead to the source reading from an unexpected
query without any warning.
Maybe we could add checks for this.
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SlideTimingSplitterEnumerator.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class SlideTimingSplitterEnumerator extends SqlSplitterEnumerator {
Review Comment:
Please add a `SlideTimingSplitterEnumeratorTest` for this class.
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SqlSplitterEnumerator.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.core.datastream.source.split.CheckpointedOffset;
+import
org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/** A sql base template split enumerator. */
+public abstract class SqlSplitterEnumerator implements SplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(SqlSplitterEnumerator.class);
Review Comment:
```suggestion
private static final Logger LOG =
LoggerFactory.getLogger(SqlSplitterEnumerator.class);
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SlideTimingSplitterEnumerator.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class SlideTimingSplitterEnumerator extends SqlSplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(SlideTimingSplitterEnumerator.class);
+
+ private final long slideStepMills;
Review Comment:
```suggestion
private final long slideStepMillis;
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SlideTimingSplitterEnumerator.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class SlideTimingSplitterEnumerator extends SqlSplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(SlideTimingSplitterEnumerator.class);
+
+ private final long slideStepMills;
+ private final long slideSpanMills;
Review Comment:
```suggestion
private final long slideSpanMillis;
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SlideTimingSplitterEnumerator.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class SlideTimingSplitterEnumerator extends SqlSplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(SlideTimingSplitterEnumerator.class);
Review Comment:
```suggestion
private static final Logger LOG =
LoggerFactory.getLogger(SlideTimingSplitterEnumerator.class);
```
##########
flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/enumerator/splitter/SlideTimingSplitterEnumerator.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.connector.jdbc.core.datastream.source.enumerator.splitter;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/** A split enumerator based on sql-parameters grains. */
+@PublicEvolving
+public class SlideTimingSplitterEnumerator extends SqlSplitterEnumerator {
+ public static final Logger LOG =
LoggerFactory.getLogger(SlideTimingSplitterEnumerator.class);
+
+ private final long slideStepMills;
+ private final long slideSpanMills;
+ private final long splitGenerateDelayMillis;
+
+ private @Nonnull Long startMills;
Review Comment:
```suggestion
private @Nonnull Long startMillis;
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]