robertwb commented on code in PR #32586:
URL: https://github.com/apache/beam/pull/32586#discussion_r1779292875
##########
sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java:
##########
@@ -121,8 +122,13 @@ protected static class JsonWriteTransform extends
SchemaTransform {
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
- WriteFilesResult<?> result =
-
input.get(INPUT_ROWS_TAG).apply(JsonIO.writeRows(configuration.getPath()).withSuffix(""));
+ // Preserve input windowing
+ JsonIO.Write<Row> writeTransform =
JsonIO.writeRows(configuration.getPath()).withSuffix("");
+ if (input.get(INPUT_ROWS_TAG).isBounded() ==
PCollection.IsBounded.UNBOUNDED) {
Review Comment:
Same.
##########
sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java:
##########
@@ -134,10 +135,15 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
if (configuration.getDelimiter() != null) {
format = format.withDelimiter(configuration.getDelimiter().charAt(0));
}
- WriteFilesResult<?> result =
- input
- .get(INPUT_ROWS_TAG)
- .apply(CsvIO.writeRows(configuration.getPath(),
format).withSuffix(""));
+
+ // Preserve input windowing
+ CsvIO.Write<Row> writeTransform =
+ CsvIO.writeRows(configuration.getPath(), format).withSuffix("");
+ if (input.get(INPUT_ROWS_TAG).isBounded() ==
PCollection.IsBounded.UNBOUNDED) {
Review Comment:
This should probably be a function of the windowing (not being global), not
of its boundedness.
--
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]