eskabetxe commented on code in PR #78:
URL: 
https://github.com/apache/flink-connector-jdbc/pull/78#discussion_r1440197286


##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/source/JdbcSourceBuilder.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.source;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import 
org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import 
org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import 
org.apache.flink.connector.jdbc.source.enumerator.SqlTemplateSplitEnumerator;
+import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.sql.ResultSet;
+
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.AUTO_COMMIT;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.READER_FETCH_BATCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_CONCURRENCY;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_FETCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_TYPE;
+
+/** A tool is used to build {@link JdbcSource} quickly. */
+@PublicEvolving
+public class JdbcSourceBuilder<OUT> {
+
+    public static final Logger LOG = 
LoggerFactory.getLogger(JdbcSourceBuilder.class);
+
+    private final Configuration configuration;
+
+    private int splitReaderFetchBatchSize;
+    private int resultSetType;
+    private int resultSetConcurrency;
+    private int resultSetFetchSize;
+    // Boolean to distinguish between default value and explicitly set 
autoCommit mode.
+    private Boolean autoCommit;
+
+    // TODO It would need a builder method to render after introducing 
streaming semantic.
+    private DeliveryGuarantee deliveryGuarantee;
+
+    private TypeInformation<OUT> typeInformation;
+
+    private final JdbcConnectionOptions.JdbcConnectionOptionsBuilder 
connOptionsBuilder;
+    private String sql;
+    private JdbcParameterValuesProvider jdbcParameterValuesProvider;
+    private ResultExtractor<OUT> resultExtractor;
+
+    private JdbcConnectionProvider connectionProvider;
+
+    JdbcSourceBuilder() {
+        this.configuration = new Configuration();
+        this.connOptionsBuilder = new 
JdbcConnectionOptions.JdbcConnectionOptionsBuilder();
+        this.splitReaderFetchBatchSize = 1024;
+        this.resultSetType = ResultSet.TYPE_FORWARD_ONLY;
+        this.resultSetConcurrency = ResultSet.CONCUR_READ_ONLY;
+        this.deliveryGuarantee = DeliveryGuarantee.NONE;
+        // Boolean to distinguish between default value and explicitly set 
autoCommit mode.
+        this.autoCommit = true;
+    }
+
+    public JdbcSourceBuilder<OUT> setSql(@Nonnull String sql) {
+        Preconditions.checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(sql), "It's required to 
set the sql.");
+        this.sql = sql;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setResultExtractor(ResultExtractor<OUT> 
resultExtractor) {
+        this.resultExtractor =
+                Preconditions.checkNotNull(resultExtractor, "resultExtractor 
must not be null.");
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setUsername(String username) {
+        Preconditions.checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(username),
+                "It's required to set the 'username'.");
+        connOptionsBuilder.withUsername(username);
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setPassword(String password) {
+        connOptionsBuilder.withPassword(password);
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setDriverName(String driverName) {
+        Preconditions.checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(driverName),
+                "It's required to set the 'driverName'.");
+        connOptionsBuilder.withDriverName(driverName);
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setDBUrl(String dbURL) {
+        Preconditions.checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(dbURL), "It's required to 
set the 'dbURL'.");
+        connOptionsBuilder.withUrl(dbURL);
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setTypeInformation(
+            @Nonnull TypeInformation<OUT> typeInformation) {
+        this.typeInformation = Preconditions.checkNotNull(typeInformation);
+        return this;
+    }
+
+    // ------ Optional 
------------------------------------------------------------------
+
+    public JdbcSourceBuilder<OUT> setJdbcParameterValuesProvider(
+            @Nonnull JdbcParameterValuesProvider parameterValuesProvider) {
+        this.jdbcParameterValuesProvider = 
Preconditions.checkNotNull(parameterValuesProvider);
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setSplitReaderFetchBatchSize(int 
splitReaderFetchBatchSize) {
+        Preconditions.checkArgument(
+                splitReaderFetchBatchSize > 0,
+                "'splitReaderFetchBatchSize' must be in range (0, %s]",
+                Integer.MAX_VALUE);
+        this.splitReaderFetchBatchSize = splitReaderFetchBatchSize;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setResultSetType(int resultSetType) {
+        this.resultSetType = resultSetType;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setResultSetConcurrency(int 
resultSetConcurrency) {
+        this.resultSetConcurrency = resultSetConcurrency;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setAutoCommit(boolean autoCommit) {
+        this.autoCommit = autoCommit;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setResultSetFetchSize(int 
resultSetFetchSize) {
+        Preconditions.checkArgument(
+                resultSetFetchSize == Integer.MIN_VALUE || resultSetFetchSize 
> 0,
+                "Illegal value %s for fetchSize, has to be positive or 
Integer.MIN_VALUE.",
+                resultSetFetchSize);
+        this.resultSetFetchSize = resultSetFetchSize;
+        return this;
+    }
+
+    public JdbcSourceBuilder<OUT> setConnectionProvider(
+            @Nonnull JdbcConnectionProvider connectionProvider) {
+        this.connectionProvider = 
Preconditions.checkNotNull(connectionProvider);
+        return this;
+    }
+
+    public JdbcSource<OUT> build() {
+        this.connectionProvider = new 
SimpleJdbcConnectionProvider(connOptionsBuilder.build());
+        if (resultSetFetchSize > 0) {
+            this.configuration.set(RESULTSET_FETCH_SIZE, resultSetFetchSize);
+        }
+        this.configuration.set(RESULTSET_CONCURRENCY, resultSetConcurrency);
+        this.configuration.set(RESULTSET_TYPE, resultSetType);
+        this.configuration.set(READER_FETCH_BATCH_SIZE, 
splitReaderFetchBatchSize);
+        this.configuration.set(AUTO_COMMIT, autoCommit);
+
+        if (StringUtils.isNullOrWhitespaceOnly(sql)) {

Review Comment:
   We are checking that sql is not null, but this is not used?



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/source/reader/extractor/ResultExtractor.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.source.reader.extractor;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.types.Row;
+
+import java.io.Serializable;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+/**
+ * The Extractor to extract the data from {@link ResultSet}.
+ *
+ * @param <T> The target data type.
+ */
+@PublicEvolving
+public interface ResultExtractor<T> extends Serializable {
+
+    /**
+     * Extract the data from the current point line of the result.
+     *
+     * @param resultSet Result set queried from a sql.
+     * @return The data object filled by the current line of the resultSet.
+     * @throws SQLException SQL exception.
+     */
+    T extract(ResultSet resultSet) throws SQLException;
+
+    /**
+     * The identifier of the extractor.
+     *
+     * @return identifier in {@link String} type.
+     */
+    default String identifier() {

Review Comment:
   do we need a "default" ?



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/source/reader/JdbcSourceSplitReader.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import 
org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import 
org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor;
+import org.apache.flink.connector.jdbc.source.split.JdbcSourceSplit;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.AUTO_COMMIT;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.READER_FETCH_BATCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_CONCURRENCY;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_FETCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_TYPE;
+
+/**
+ * The JDBC source reader to read data from jdbc splits.
+ *
+ * @param <T> The type of the record read from the source.
+ */
+public class JdbcSourceSplitReader<T>
+        implements SplitReader<RecordAndOffset<T>, JdbcSourceSplit>, 
ResultTypeQueryable<T> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(JdbcSourceSplitReader.class);
+
+    private final Configuration config;
+    @Nullable private JdbcSourceSplit currentSplit;
+    private final Queue<JdbcSourceSplit> splits;
+    private final TypeInformation<T> typeInformation;
+    private final JdbcConnectionProvider connectionProvider;
+    private transient Connection connection;
+    private transient PreparedStatement statement;
+    private transient ResultSet resultSet;
+
+    private final ResultExtractor<T> resultExtractor;
+    protected boolean hasNextRecordCurrentSplit;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private final int splitReaderFetchBatchSize;
+
+    private final int resultSetType;
+    private final int resultSetConcurrency;
+    private final int resultSetFetchSize;
+    // Boolean to distinguish between default value and explicitly set 
autoCommit mode.
+    private final Boolean autoCommit;
+    private int currentSplitOffset;
+
+    private final SourceReaderContext context;
+
+    public JdbcSourceSplitReader(
+            SourceReaderContext context,
+            Configuration config,
+            TypeInformation<T> typeInformation,
+            JdbcConnectionProvider connectionProvider,
+            DeliveryGuarantee deliveryGuarantee,
+            ResultExtractor<T> resultExtractor) {
+        this.context = Preconditions.checkNotNull(context);
+        this.config = Preconditions.checkNotNull(config);
+        this.typeInformation = Preconditions.checkNotNull(typeInformation);
+        this.connectionProvider = 
Preconditions.checkNotNull(connectionProvider);
+        this.resultSetType = config.getInteger(RESULTSET_TYPE);
+        this.resultSetConcurrency = config.getInteger(RESULTSET_CONCURRENCY);
+        this.resultSetFetchSize = config.getInteger(RESULTSET_FETCH_SIZE);
+        this.autoCommit = config.getBoolean(AUTO_COMMIT);
+        this.deliveryGuarantee = Preconditions.checkNotNull(deliveryGuarantee);
+        this.splits = new ArrayDeque<>();
+        this.hasNextRecordCurrentSplit = false;
+        this.currentSplit = null;
+        int splitReaderFetchBatchSize = 
config.getInteger(READER_FETCH_BATCH_SIZE);
+        Preconditions.checkArgument(
+                splitReaderFetchBatchSize > 0 && splitReaderFetchBatchSize < 
Integer.MAX_VALUE);
+        this.splitReaderFetchBatchSize = splitReaderFetchBatchSize;
+        this.resultExtractor = Preconditions.checkNotNull(resultExtractor);
+        this.currentSplitOffset = 0;
+    }
+
+    @Override
+    public RecordsWithSplitIds<RecordAndOffset<T>> fetch() throws IOException {
+
+        checkSplitOrStartNext();
+
+        if (!hasNextRecordCurrentSplit) {
+            return finishSplit();
+        }
+
+        RecordsBySplits.Builder<RecordAndOffset<T>> recordAndOffsetBuilder =
+                new RecordsBySplits.Builder<>();
+        Preconditions.checkState(currentSplit != null, "currentSplit");
+        int batch = this.splitReaderFetchBatchSize;
+        while (batch > 0 && hasNextRecordCurrentSplit) {
+            try {
+                T ele = resultExtractor.extract(resultSet);
+                recordAndOffsetBuilder.add(
+                        currentSplit, new RecordAndOffset<>(ele, 
++currentSplitOffset, 0));
+                batch--;
+                hasNextRecordCurrentSplit = resultSet.next();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+        if (!hasNextRecordCurrentSplit) {
+            currentSplitOffset = 0;
+            recordAndOffsetBuilder.addFinishedSplit(currentSplit.splitId());
+            closeResultSetAndStatement();
+        }
+        return recordAndOffsetBuilder.build();
+    }
+
+    private RecordsWithSplitIds<RecordAndOffset<T>> finishSplit() {
+
+        closeResultSetAndStatement();
+
+        RecordsBySplits.Builder<RecordAndOffset<T>> builder = new 
RecordsBySplits.Builder<>();
+        Preconditions.checkState(currentSplit != null, "currentSplit");
+        builder.addFinishedSplit(currentSplit.splitId());
+        currentSplit = null;
+        return builder.build();
+    }
+
+    private void closeResultSetAndStatement() {
+        try {
+            if (resultSet != null && !resultSet.isClosed()) {
+                resultSet.close();
+            }
+            if (statement != null && !statement.isClosed()) {
+                statement.close();
+            }
+            resultSet = null;
+            statement = null;
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<JdbcSourceSplit> 
splitsChanges) {
+        if (!(splitsChanges instanceof SplitsAddition)) {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "The SplitChange type of %s is not supported.",
+                            splitsChanges.getClass()));
+        }
+
+        LOG.debug("Handling split change {}", splitsChanges);
+        splits.addAll(splitsChanges.splits());
+    }
+
+    @Override
+    public void wakeUp() {}
+
+    @Override
+    public void close() throws Exception {
+        closeResultSetAndStatement();
+        if (connection != null && !connection.isClosed()) {
+            connection.close();
+        }
+        connection = null;
+        currentSplit = null;
+    }
+
+    @Override
+    public TypeInformation<T> getProducedType() {
+        return typeInformation;
+    }
+
+    private void checkSplitOrStartNext() {
+
+        try {
+            if (hasNextRecordCurrentSplit && resultSet != null) {
+                return;
+            }
+
+            final JdbcSourceSplit nextSplit = splits.poll();
+            if (nextSplit == null) {
+                throw new IOException("Cannot fetch from another split - no 
split remaining");
+            }
+            currentSplit = nextSplit;
+            openResultSetForSplit(currentSplit);
+        } catch (SQLException | ClassNotFoundException | IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void discardSplit(JdbcSourceSplit split) throws SQLException {
+        if (split.getOffset() != 0) {
+            hasNextRecordCurrentSplit = false;
+            currentSplitOffset = 0;
+            if (resultSet != null && !resultSet.isClosed()) {
+                resultSet.close();
+            }
+            if (statement != null && !statement.isClosed()) {
+                statement.close();
+            }
+            resultSet = null;
+            statement = null;
+            currentSplit = null;
+        }
+    }
+
+    private void reOpenConnectionIfNeeded() throws SQLException, 
ClassNotFoundException {

Review Comment:
   getOrEstablishConnection ?



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/source/JdbcSource.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.source;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import 
org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import 
org.apache.flink.connector.jdbc.source.enumerator.JdbcSourceEnumStateSerializer;
+import org.apache.flink.connector.jdbc.source.enumerator.JdbcSourceEnumerator;
+import 
org.apache.flink.connector.jdbc.source.enumerator.JdbcSourceEnumeratorState;
+import 
org.apache.flink.connector.jdbc.source.enumerator.JdbcSqlSplitEnumeratorBase;
+import org.apache.flink.connector.jdbc.source.reader.JdbcSourceReader;
+import org.apache.flink.connector.jdbc.source.reader.JdbcSourceSplitReader;
+import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor;
+import org.apache.flink.connector.jdbc.source.split.JdbcSourceSplit;
+import org.apache.flink.connector.jdbc.source.split.JdbcSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Objects;
+
+/** JDBC source. */
+@PublicEvolving
+public class JdbcSource<OUT>
+        implements Source<OUT, JdbcSourceSplit, JdbcSourceEnumeratorState>,
+                ResultTypeQueryable<OUT> {
+
+    private final Boundedness boundedness;
+    private final TypeInformation<OUT> typeInformation;
+
+    private final Configuration configuration;
+    private final JdbcSqlSplitEnumeratorBase.Provider<JdbcSourceSplit> 
sqlSplitEnumeratorProvider;
+
+    protected JdbcConnectionProvider connectionProvider;
+    private final ResultExtractor<OUT> resultExtractor;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    JdbcSource(
+            Configuration configuration,
+            JdbcConnectionProvider connectionProvider,
+            JdbcSqlSplitEnumeratorBase.Provider<JdbcSourceSplit> 
sqlSplitEnumeratorProvider,
+            ResultExtractor<OUT> resultExtractor,
+            @Nullable TypeInformation<OUT> typeInformation,
+            @Nullable DeliveryGuarantee deliveryGuarantee) {

Review Comment:
   Should we create another constructor without this two vars, and here expect 
that are not null?



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/source/reader/JdbcSourceSplitReader.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.base.source.reader.RecordsBySplits;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import 
org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import 
org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor;
+import org.apache.flink.connector.jdbc.source.split.JdbcSourceSplit;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.AUTO_COMMIT;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.READER_FETCH_BATCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_CONCURRENCY;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_FETCH_SIZE;
+import static 
org.apache.flink.connector.jdbc.source.JdbcSourceOptions.RESULTSET_TYPE;
+
+/**
+ * The JDBC source reader to read data from jdbc splits.
+ *
+ * @param <T> The type of the record read from the source.
+ */
+public class JdbcSourceSplitReader<T>
+        implements SplitReader<RecordAndOffset<T>, JdbcSourceSplit>, 
ResultTypeQueryable<T> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(JdbcSourceSplitReader.class);
+
+    private final Configuration config;
+    @Nullable private JdbcSourceSplit currentSplit;
+    private final Queue<JdbcSourceSplit> splits;
+    private final TypeInformation<T> typeInformation;
+    private final JdbcConnectionProvider connectionProvider;
+    private transient Connection connection;
+    private transient PreparedStatement statement;
+    private transient ResultSet resultSet;
+
+    private final ResultExtractor<T> resultExtractor;
+    protected boolean hasNextRecordCurrentSplit;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private final int splitReaderFetchBatchSize;
+
+    private final int resultSetType;
+    private final int resultSetConcurrency;
+    private final int resultSetFetchSize;
+    // Boolean to distinguish between default value and explicitly set 
autoCommit mode.
+    private final Boolean autoCommit;
+    private int currentSplitOffset;
+
+    private final SourceReaderContext context;
+
+    public JdbcSourceSplitReader(
+            SourceReaderContext context,
+            Configuration config,
+            TypeInformation<T> typeInformation,
+            JdbcConnectionProvider connectionProvider,
+            DeliveryGuarantee deliveryGuarantee,
+            ResultExtractor<T> resultExtractor) {
+        this.context = Preconditions.checkNotNull(context);
+        this.config = Preconditions.checkNotNull(config);
+        this.typeInformation = Preconditions.checkNotNull(typeInformation);
+        this.connectionProvider = 
Preconditions.checkNotNull(connectionProvider);
+        this.resultSetType = config.getInteger(RESULTSET_TYPE);
+        this.resultSetConcurrency = config.getInteger(RESULTSET_CONCURRENCY);
+        this.resultSetFetchSize = config.getInteger(RESULTSET_FETCH_SIZE);
+        this.autoCommit = config.getBoolean(AUTO_COMMIT);
+        this.deliveryGuarantee = Preconditions.checkNotNull(deliveryGuarantee);
+        this.splits = new ArrayDeque<>();
+        this.hasNextRecordCurrentSplit = false;
+        this.currentSplit = null;
+        int splitReaderFetchBatchSize = 
config.getInteger(READER_FETCH_BATCH_SIZE);
+        Preconditions.checkArgument(
+                splitReaderFetchBatchSize > 0 && splitReaderFetchBatchSize < 
Integer.MAX_VALUE);
+        this.splitReaderFetchBatchSize = splitReaderFetchBatchSize;
+        this.resultExtractor = Preconditions.checkNotNull(resultExtractor);
+        this.currentSplitOffset = 0;
+    }
+
+    @Override
+    public RecordsWithSplitIds<RecordAndOffset<T>> fetch() throws IOException {
+
+        checkSplitOrStartNext();
+
+        if (!hasNextRecordCurrentSplit) {
+            return finishSplit();
+        }
+
+        RecordsBySplits.Builder<RecordAndOffset<T>> recordAndOffsetBuilder =
+                new RecordsBySplits.Builder<>();
+        Preconditions.checkState(currentSplit != null, "currentSplit");
+        int batch = this.splitReaderFetchBatchSize;
+        while (batch > 0 && hasNextRecordCurrentSplit) {
+            try {
+                T ele = resultExtractor.extract(resultSet);

Review Comment:
   ele -> record



-- 
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]

Reply via email to