@@ -1949,6 +1949,8 @@ public <V extends JdbcWriteResult> WriteWithResults<T, V> withWriteResults(
1949
1949
.setStatement (inner .getStatement ())
1950
1950
.setTable (inner .getTable ())
1951
1951
.setAutoSharding (inner .getAutoSharding ())
1952
+ .setBatchSize (inner .getBatchSize ())
1953
+ .setMaxBatchBufferingDuration (inner .getMaxBatchBufferingDuration ())
1952
1954
.build ();
1953
1955
}
1954
1956
@@ -2055,6 +2057,10 @@ public abstract static class WriteWithResults<T, V extends JdbcWriteResult>
2055
2057
2056
2058
abstract @ Nullable RowMapper <V > getRowMapper ();
2057
2059
2060
+ abstract @ Nullable Long getBatchSize ();
2061
+
2062
+ abstract @ Nullable Long getMaxBatchBufferingDuration ();
2063
+
2058
2064
abstract Builder <T , V > toBuilder ();
2059
2065
2060
2066
@ AutoValue .Builder
@@ -2064,6 +2070,10 @@ abstract Builder<T, V> setDataSourceProviderFn(
2064
2070
2065
2071
abstract Builder <T , V > setAutoSharding (@ Nullable Boolean autoSharding );
2066
2072
2073
+ abstract Builder <T , V > setBatchSize (@ Nullable Long batchSize );
2074
+
2075
+ abstract Builder <T , V > setMaxBatchBufferingDuration (@ Nullable Long maxBatchBufferingDuration );
2076
+
2067
2077
abstract Builder <T , V > setStatement (@ Nullable ValueProvider <String > statement );
2068
2078
2069
2079
abstract Builder <T , V > setPreparedStatementSetter (
@@ -2080,6 +2090,19 @@ abstract Builder<T, V> setPreparedStatementSetter(
2080
2090
abstract WriteWithResults <T , V > build ();
2081
2091
}
2082
2092
2093
+ public WriteWithResults <T , V > withBatchSize (long batchSize ) {
2094
+ checkArgument (batchSize > 0 , "batchSize must be > 0, but was %s" , batchSize );
2095
+ return toBuilder ().setBatchSize (batchSize ).build ();
2096
+ }
2097
+
2098
+ public WriteWithResults <T , V > withMaxBatchBufferingDuration (long maxBatchBufferingDuration ) {
2099
+ checkArgument (
2100
+ maxBatchBufferingDuration > 0 ,
2101
+ "maxBatchBufferingDuration must be > 0, but was %s" ,
2102
+ maxBatchBufferingDuration );
2103
+ return toBuilder ().setMaxBatchBufferingDuration (maxBatchBufferingDuration ).build ();
2104
+ }
2105
+
2083
2106
public WriteWithResults <T , V > withDataSourceConfiguration (DataSourceConfiguration config ) {
2084
2107
return withDataSourceProviderFn (new DataSourceProviderFromDataSourceConfiguration (config ));
2085
2108
}
@@ -2173,9 +2196,16 @@ public PCollection<V> expand(PCollection<T> input) {
2173
2196
autoSharding == null || (autoSharding && input .isBounded () != IsBounded .UNBOUNDED ),
2174
2197
"Autosharding is only supported for streaming pipelines." );
2175
2198
2199
+ Long batchSizeAsLong = getBatchSize ();
2200
+ long batchSize = batchSizeAsLong == null ? DEFAULT_BATCH_SIZE : batchSizeAsLong ;
2201
+ Long maxBufferingDurationAsLong = getMaxBatchBufferingDuration ();
2202
+ long maxBufferingDuration =
2203
+ maxBufferingDurationAsLong == null
2204
+ ? DEFAULT_MAX_BATCH_BUFFERING_DURATION
2205
+ : maxBufferingDurationAsLong ;
2206
+
2176
2207
PCollection <Iterable <T >> iterables =
2177
- JdbcIO .<T >batchElements (
2178
- input , autoSharding , DEFAULT_BATCH_SIZE , DEFAULT_MAX_BATCH_BUFFERING_DURATION );
2208
+ JdbcIO .<T >batchElements (input , autoSharding , batchSize , maxBufferingDuration );
2179
2209
return iterables .apply (
2180
2210
ParDo .of (
2181
2211
new WriteFn <T , V >(
@@ -2187,8 +2217,8 @@ public PCollection<V> expand(PCollection<T> input) {
2187
2217
.setStatement (getStatement ())
2188
2218
.setRetryConfiguration (getRetryConfiguration ())
2189
2219
.setReturnResults (true )
2190
- .setBatchSize (1L )
2191
- .setMaxBatchBufferingDuration (DEFAULT_MAX_BATCH_BUFFERING_DURATION )
2220
+ .setBatchSize (1L ) // We are writing iterables 1 at a time.
2221
+ .setMaxBatchBufferingDuration (maxBufferingDuration )
2192
2222
.build ())));
2193
2223
}
2194
2224
}
0 commit comments