claudevdm commented on code in PR #34417: URL: https://github.com/apache/beam/pull/34417#discussion_r2012496836
########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = + (config.getReadQuery() != null && !"".equals(config.getReadQuery())); Review Comment: Can we use !config.getReadQuery().isEmpty() instead of !"".equals(config.getReadQuery())? ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = Review Comment: Oh nvm, I see you already did this. You can remove all validation logic from expand(). It is already implemented in validate()? ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = + (config.getReadQuery() != null && !"".equals(config.getReadQuery())); + boolean locationPresent = (config.getLocation() != null && !"".equals(config.getLocation())); + + if (readQueryPresent && locationPresent) { + throw new IllegalArgumentException("Query and Table are mutually exclusive configurations"); } - Boolean outputParallelization = config.getOutputParallelization(); - if (outputParallelization != null) { - readRows = readRows.withOutputParallelization(outputParallelization); + if (!readQueryPresent && !locationPresent) { + throw new IllegalArgumentException("Either Query or Table must be specified."); } - Boolean disableAutoCommit = config.getDisableAutoCommit(); - if (disableAutoCommit != null) { - readRows = readRows.withDisableAutoCommit(disableAutoCommit); + + @Nullable String partitionColumn = config.getPartitionColumn(); + @Nullable String location = config.getLocation(); + if (partitionColumn != null) { + JdbcIO.ReadWithPartitions<Row, ?> readRowsWithParitions = + JdbcIO.<Row>readWithPartitions() + .withDataSourceConfiguration(dataSourceConfiguration()) + .withTable(location) + .withPartitionColumn(partitionColumn) + .withRowOutput(); + + @Nullable Integer partitions = config.getNumPartitions(); + if (partitions != null) { + readRowsWithParitions = readRowsWithParitions.withNumPartitions(partitions); + } + + @Nullable Integer fetchSize = config.getFetchSize(); + if (fetchSize != null && fetchSize > 0) { + readRowsWithParitions = readRowsWithParitions.withFetchSize(fetchSize); + } + + @Nullable Boolean disableAutoCommit = config.getDisableAutoCommit(); + if (disableAutoCommit != null) { + readRowsWithParitions = readRowsWithParitions.withDisableAutoCommit(disableAutoCommit); + } + return PCollectionRowTuple.of("output", input.getPipeline().apply(readRowsWithParitions)); + } else { Review Comment: This else block is unnecessary, can we remove it and unindent the remaining code? ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = Review Comment: @shubham-j-sde , can you move validation logic to the validate() method? https://github.com/apache/beam/blob/a82090bb9a416472c0f48a4e2467c6e5f96568e2/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java#L377 ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = + (config.getReadQuery() != null && !"".equals(config.getReadQuery())); + boolean locationPresent = (config.getLocation() != null && !"".equals(config.getLocation())); + + if (readQueryPresent && locationPresent) { + throw new IllegalArgumentException("Query and Table are mutually exclusive configurations"); } - Boolean outputParallelization = config.getOutputParallelization(); - if (outputParallelization != null) { - readRows = readRows.withOutputParallelization(outputParallelization); + if (!readQueryPresent && !locationPresent) { + throw new IllegalArgumentException("Either Query or Table must be specified."); } - Boolean disableAutoCommit = config.getDisableAutoCommit(); - if (disableAutoCommit != null) { - readRows = readRows.withDisableAutoCommit(disableAutoCommit); + + @Nullable String partitionColumn = config.getPartitionColumn(); + @Nullable String location = config.getLocation(); + if (partitionColumn != null) { + JdbcIO.ReadWithPartitions<Row, ?> readRowsWithParitions = Review Comment: The default readWithPartitions being called here defaults to a long partition column type https://github.com/apache/beam/blob/4a7f19d997395264aaed32adde544fa340ce6124/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java#L406 If we want the user to be able to partition by datetime column types, we will need to have them pass the partition column type as a parameter too, and call eadWithPartitions(partitioningColumnType) https://github.com/apache/beam/blob/4a7f19d997395264aaed32adde544fa340ce6124/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java#L378 @svetakvsundhar is the goal to support just long partitions, or both long and date? ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = Review Comment: Actually, there is already a validate() function defined in this class. Lets add new validation logic there. Specifically, if there is a partition column, then we need to check that the table parameter is set, but not the query parameter ########## sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java: ########## @@ -213,25 +214,67 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - String query = config.getReadQuery(); - if (query == null) { - query = String.format("SELECT * FROM %s", config.getLocation()); - } - JdbcIO.ReadRows readRows = - JdbcIO.readRows().withDataSourceConfiguration(dataSourceConfiguration()).withQuery(query); - Integer fetchSize = config.getFetchSize(); - if (fetchSize != null && fetchSize > 0) { - readRows = readRows.withFetchSize(fetchSize); + + boolean readQueryPresent = + (config.getReadQuery() != null && !"".equals(config.getReadQuery())); + boolean locationPresent = (config.getLocation() != null && !"".equals(config.getLocation())); + + if (readQueryPresent && locationPresent) { + throw new IllegalArgumentException("Query and Table are mutually exclusive configurations"); } - Boolean outputParallelization = config.getOutputParallelization(); - if (outputParallelization != null) { - readRows = readRows.withOutputParallelization(outputParallelization); + if (!readQueryPresent && !locationPresent) { + throw new IllegalArgumentException("Either Query or Table must be specified."); } - Boolean disableAutoCommit = config.getDisableAutoCommit(); - if (disableAutoCommit != null) { - readRows = readRows.withDisableAutoCommit(disableAutoCommit); + + @Nullable String partitionColumn = config.getPartitionColumn(); + @Nullable String location = config.getLocation(); + if (partitionColumn != null) { + JdbcIO.ReadWithPartitions<Row, ?> readRowsWithParitions = Review Comment: @shubham-j-sde, @svetakvsundhar if we are not adding support for date partitions in this PR, we should document that only integer partition columns are supported -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org