[
https://issues.apache.org/jira/browse/BEAM-13184?focusedWorklogId=680879&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-680879
]
ASF GitHub Bot logged work on BEAM-13184:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 12/Nov/21 16:05
Start Date: 12/Nov/21 16:05
Worklog Time Spent: 10m
Work Description: aromanenko-dev commented on a change in pull request
#15863:
URL: https://github.com/apache/beam/pull/15863#discussion_r748398715
##########
File path:
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
##########
@@ -1494,8 +1509,49 @@ void set(
checkArgument(
(getDataSourceProviderFn() != null),
"withDataSourceConfiguration() or withDataSourceProviderFn() is
required");
-
- return input.apply(
+ checkArgument(
+ ((getAutoSharding() == null || !getAutoSharding())
+ && input.isBounded() == IsBounded.BOUNDED)
+ || input.isBounded() == IsBounded.UNBOUNDED,
+ "Autosharding is only supported for streaming pipelines.");
+ ;
+
+ PCollection<Iterable<T>> iterables;
+ if (input.isBounded() == IsBounded.UNBOUNDED
+ && getAutoSharding() != null
+ && getAutoSharding()) {
+ iterables =
+ input
+ .apply(WithKeys.<String, T>of(""))
Review comment:
Why `of("")` ?
##########
File path:
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
##########
@@ -1494,8 +1509,49 @@ void set(
checkArgument(
(getDataSourceProviderFn() != null),
"withDataSourceConfiguration() or withDataSourceProviderFn() is
required");
-
- return input.apply(
+ checkArgument(
+ ((getAutoSharding() == null || !getAutoSharding())
+ && input.isBounded() == IsBounded.BOUNDED)
+ || input.isBounded() == IsBounded.UNBOUNDED,
+ "Autosharding is only supported for streaming pipelines.");
+ ;
+
+ PCollection<Iterable<T>> iterables;
+ if (input.isBounded() == IsBounded.UNBOUNDED
+ && getAutoSharding() != null
+ && getAutoSharding()) {
+ iterables =
+ input
+ .apply(WithKeys.<String, T>of(""))
+ .apply(
+ GroupIntoBatches.<String, T>ofSize(DEFAULT_BATCH_SIZE)
+ .withMaxBufferingDuration(Duration.millis(200))
+ .withShardedKey())
+ .apply(Values.create());
+ } else {
+ iterables =
Review comment:
Why this branch is needed? To batch per bundle for bounded case?
##########
File path:
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
##########
@@ -1494,8 +1509,49 @@ void set(
checkArgument(
(getDataSourceProviderFn() != null),
"withDataSourceConfiguration() or withDataSourceProviderFn() is
required");
-
- return input.apply(
+ checkArgument(
+ ((getAutoSharding() == null || !getAutoSharding())
Review comment:
A bit simpler condition:
```
checkArgument(getAutoSharding() != null && getAutoSharding() &&
input.isBounded() != IsBounded.UNBOUNDED)
```
##########
File path:
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
##########
@@ -1494,8 +1509,49 @@ void set(
checkArgument(
(getDataSourceProviderFn() != null),
"withDataSourceConfiguration() or withDataSourceProviderFn() is
required");
-
- return input.apply(
+ checkArgument(
+ ((getAutoSharding() == null || !getAutoSharding())
+ && input.isBounded() == IsBounded.BOUNDED)
+ || input.isBounded() == IsBounded.UNBOUNDED,
+ "Autosharding is only supported for streaming pipelines.");
+ ;
+
+ PCollection<Iterable<T>> iterables;
+ if (input.isBounded() == IsBounded.UNBOUNDED
+ && getAutoSharding() != null
+ && getAutoSharding()) {
+ iterables =
+ input
+ .apply(WithKeys.<String, T>of(""))
+ .apply(
+ GroupIntoBatches.<String, T>ofSize(DEFAULT_BATCH_SIZE)
+ .withMaxBufferingDuration(Duration.millis(200))
+ .withShardedKey())
+ .apply(Values.create());
+ } else {
+ iterables =
+ input.apply(
+ ParDo.of(
+ new DoFn<T, Iterable<T>>() {
+ List<T> outputList;
+
+ @ProcessElement
+ public void process(ProcessContext c) {
+ if (outputList == null) {
+ outputList = new ArrayList<>();
+ }
+ outputList.add(c.element());
+ }
+
+ @FinishBundle
+ public void finish(FinishBundleContext c) {
+ System.out.println("List size is " +
outputList.size());
Review comment:
Please, remove it or add to logs.
##########
File path:
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
##########
@@ -1432,6 +1442,11 @@ void set(
return toBuilder().setPreparedStatementSetter(setter).build();
}
+ /** If true, enables using a dynamically determined number of shards to
write. */
+ public WriteWithResults<T, V> withAutoSharding() {
Review comment:
Should we allow to customise a sharding function?
--
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 680879)
Time Spent: 1.5h (was: 1h 20m)
> Support autosharding for JdbcIO writers
> ---------------------------------------
>
> Key: BEAM-13184
> URL: https://issues.apache.org/jira/browse/BEAM-13184
> Project: Beam
> Issue Type: Task
> Components: io-java-jdbc
> Reporter: Pablo Estrada
> Priority: P2
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> This should improve efficiency for Jdbc writers on streaming pipelines
--
This message was sent by Atlassian Jira
(v8.20.1#820001)