1996fanrui commented on code in PR #119: URL: https://github.com/apache/flink-connector-jdbc/pull/119#discussion_r1628865178
########## docs/content/docs/connectors/datastream/jdbc.md: ########## @@ -26,16 +26,211 @@ under the License. # JDBC Connector -This connector provides a sink that writes data to a JDBC database. +This connector provides a source that read data from a JDBC database and +provides a sink that writes data to a JDBC database. To use it, add the following dependency to your project (along with your JDBC driver): {{< connector_artifact flink-connector-jdbc jdbc >}} -Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). -A driver dependency is also required to connect to a specified database. Please consult your database documentation on how to add the corresponding driver. +Note that the streaming connectors are currently __NOT__ part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). +A driver dependency is also required to connect to a specified database. +Please consult your database documentation on how to add the corresponding driver. -## `JdbcSink.sink` +## Source of JDBC Connector(Experimental) Review Comment: ```suggestion ## Source of JDBC Connector(Experimental) ``` This part is not needed. ########## docs/content/docs/connectors/datastream/jdbc.md: ########## @@ -26,16 +26,211 @@ under the License. # JDBC Connector -This connector provides a sink that writes data to a JDBC database. +This connector provides a source that read data from a JDBC database and +provides a sink that writes data to a JDBC database. To use it, add the following dependency to your project (along with your JDBC driver): {{< connector_artifact flink-connector-jdbc jdbc >}} -Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). -A driver dependency is also required to connect to a specified database. Please consult your database documentation on how to add the corresponding driver. +Note that the streaming connectors are currently __NOT__ part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). +A driver dependency is also required to connect to a specified database. +Please consult your database documentation on how to add the corresponding driver. -## `JdbcSink.sink` +## Source of JDBC Connector(Experimental) + +Configuration goes as follow (see also {{< javadoc file="org/apache/flink/connector/jdbc/source/JdbcSource.html" name="JdbcSource javadoc" >}} +and {{< javadoc file="org/apache/flink/connector/jdbc/source/JdbcSourceBuilder.html" name="JdbcSourceBuilder javadoc" >}}). + +### `JdbcSource.builder` + +{{< tabs "4ab65f13-607a-411a-8d24-e709f701cd41" >}} +{{< tab "Java" >}} +```java +JdbcSource source = JdbcSourceBuilder.builder() + // Required + .setSql(...) + .setResultExtractor(...) + .setUsername(...) + .setPassword(...) + .setDriverName(...) + .setDBUrl(...) + .setTypeInformation(...) + + // Optional + .setContinuousUnBoundingSettings(...) + .setJdbcParameterValuesProvider(...) + .setDeliveryGuarantee(...) + .setConnectionCheckTimeoutSeconds(...) + + // The extended JDBC connection property passing + .setConnectionProperty("key", "value") + + // other attributes + .setSplitReaderFetchBatchSize(...) + .setResultSetType(...) + .setResultSetConcurrency(...) + .setAutoCommit(...) + .setResultSetFetchSize(...) + .setConnectionProvider(...) + .build(); + +``` +{{< /tab >}} +{{< tab "Python" >}} +```python +Still not supported in Python API. +``` +{{< /tab >}} +{{< /tabs >}} + +#### Delivery guarantee + +The JDBC source provides `at-least-once`/`at-most-once(default)`/`exactly-once` guarantee. +The `JdbcSource` supports `Delivery guarantee` semantic based on `Concur` of `ResultSet`. + +**NOTE:** Here's a few disadvantage. It only makes sense for corresponding semantic +that the `ResultSet` corresponding to this SQL(`JdbcSourceSplit`) +remains unchanged in the whole lifecycle of `JdbcSourceSplit` processing. +Unfortunately, this condition is not met in most databases and data scenarios. +See [FLIP-239](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217386271) for more details. + +#### ResultExtractor + +An `Extractor` to extract a record from `ResultSet` executed by a sql. + +```java +import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor; + +import java.sql.ResultSet; +import java.sql.SQLException; + +class Book { + public Book(Long id, String title) { + this.id = id; + this.title = title; + } + + final Long id; + final String title; +}; + +ResultExtractor resultExtractor = new ResultExtractor() { + @Override + public Object extract(ResultSet resultSet) throws SQLException { + return new Book(resultSet.getLong("id"), resultSet.getString("titile")); + } +}; + +``` + +#### JdbcParameterValuesProvider + +A provider to provide parameters in sql to fulfill actual value in the corresponding placeholders, which is in the form of two-dimension array. +See {{< javadoc file="org/apache/flink/connector/jdbc/split/JdbcParameterValuesProvider.html" name="JdbcParameterValuesProvider javadoc" >}} for more details. + +```java + +class TestEntry { + ... +}; + +ResultSetExtractor extractor = ...; + +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +JdbcSource<TestEntry> jdbcSource = + JdbcSource.<TestEntry>builder() + .setTypeInformation(TypeInformation.of(TestEntry.class)) + .setSql("select * from testing_table where id >= ? and id <= ?") + .setJdbcParameterValuesProvider( + new JdbcGenericParameterValuesProvider( + new Serializable[][] {{1001, 1005}, {1006, 1010}})) + ... + .build(); +env.fromSource(jdbcSource, WatermarkStrategy.noWatermarks(), "TestSource") + .addSink(new DiscardSink()); +env.execute(); + +``` + +#### Minimalist Streaming Semantic and ContinuousUnBoundingSettings +If you want to generate continuous milliseconds parameters based on sliding-window, Review Comment: ```suggestion If you want to generate continuous milliseconds parameters based on sliding-window, ``` Add the empty line. ########## docs/content/docs/connectors/datastream/jdbc.md: ########## @@ -26,16 +26,211 @@ under the License. # JDBC Connector -This connector provides a sink that writes data to a JDBC database. +This connector provides a source that read data from a JDBC database and +provides a sink that writes data to a JDBC database. To use it, add the following dependency to your project (along with your JDBC driver): {{< connector_artifact flink-connector-jdbc jdbc >}} -Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). -A driver dependency is also required to connect to a specified database. Please consult your database documentation on how to add the corresponding driver. +Note that the streaming connectors are currently __NOT__ part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). +A driver dependency is also required to connect to a specified database. +Please consult your database documentation on how to add the corresponding driver. -## `JdbcSink.sink` +## Source of JDBC Connector(Experimental) + +Configuration goes as follow (see also {{< javadoc file="org/apache/flink/connector/jdbc/source/JdbcSource.html" name="JdbcSource javadoc" >}} +and {{< javadoc file="org/apache/flink/connector/jdbc/source/JdbcSourceBuilder.html" name="JdbcSourceBuilder javadoc" >}}). + +### `JdbcSource.builder` Review Comment: ```suggestion ## Jdbc Source ``` Jdbc Source can be as the second level instead of the third level. You can refer Kafka connector doc[1]. `Kafka Source`, `Kafka SourceFunction` and `Kafka Producer` as the second level. We don't need to distinguish the source or sink. (Current level is a little fine-grained.) [1] https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/kafka/#kafka-source -- 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