Skip to content

Commit 489bd7a

Browse files
author
Michel Davit
authored
Fix file batch operation (#5519)
1 parent 0a450e9 commit 489bd7a

File tree

3 files changed

+74
-51
lines changed

3 files changed

+74
-51
lines changed

scio-core/src/main/java/com/spotify/scio/transforms/FileDownloadDoFn.java

Lines changed: 41 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,15 @@
2020
import com.spotify.scio.util.RemoteFileUtil;
2121
import java.net.URI;
2222
import java.nio.file.Path;
23-
import java.util.ArrayList;
24-
import java.util.Iterator;
25-
import java.util.List;
23+
import java.util.*;
24+
import java.util.function.Consumer;
2625
import java.util.stream.Collectors;
2726
import org.apache.beam.sdk.transforms.DoFn;
2827
import org.apache.beam.sdk.transforms.SerializableFunction;
2928
import org.apache.beam.sdk.transforms.display.DisplayData;
3029
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
30+
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
31+
import org.apache.beam.sdk.values.ValueInSingleWindow;
3132
import org.joda.time.Instant;
3233
import org.slf4j.Logger;
3334
import org.slf4j.LoggerFactory;
@@ -37,7 +38,7 @@ public class FileDownloadDoFn<OutputT> extends DoFn<URI, OutputT> {
3738

3839
private static final Logger LOG = LoggerFactory.getLogger(FileDownloadDoFn.class);
3940

40-
private final List<Element> batch;
41+
private final List<ValueInSingleWindow<URI>> batch;
4142
private final RemoteFileUtil remoteFileUtil;
4243
private final SerializableFunction<Path, OutputT> fn;
4344
private final int batchSize;
@@ -78,21 +79,37 @@ public void startBundle(StartBundleContext context) {
7879
this.batch.clear();
7980
}
8081

82+
// kept for binary compatibility. Must not be used
83+
// TODO: remove in 0.15.0
84+
@Deprecated
85+
public void processElement(
86+
URI element, Instant timestamp, OutputReceiver<OutputT> out, BoundedWindow window) {
87+
processElement(element, timestamp, window, null, out);
88+
}
89+
8190
@ProcessElement
8291
public void processElement(
8392
@DoFn.Element URI element,
8493
@Timestamp Instant timestamp,
85-
OutputReceiver<OutputT> out,
86-
BoundedWindow window) {
87-
batch.add(new Element(element, timestamp, window));
94+
BoundedWindow window,
95+
PaneInfo pane,
96+
OutputReceiver<OutputT> out) {
97+
batch.add(ValueInSingleWindow.of(element, timestamp, window, pane));
8898
if (batch.size() >= batchSize) {
89-
processBatch(out);
99+
flush(
100+
r -> {
101+
final OutputT o = r.getValue();
102+
final Instant ts = r.getTimestamp();
103+
final Collection<BoundedWindow> ws = Collections.singleton(r.getWindow());
104+
final PaneInfo p = r.getPane();
105+
out.outputWindowedValue(o, ts, ws, p);
106+
});
90107
}
91108
}
92109

93110
@FinishBundle
94111
public void finishBundle(FinishBundleContext context) {
95-
processBatch(context);
112+
flush(p -> context.output(p.getValue(), p.getTimestamp(), p.getWindow()));
96113
}
97114

98115
@Override
@@ -103,51 +120,31 @@ public void populateDisplayData(DisplayData.Builder builder) {
103120
.add(DisplayData.item("Keep Downloaded Files", keep));
104121
}
105122

106-
private void processBatch(OutputReceiver<OutputT> outputReceiver) {
123+
private void flush(Consumer<ValueInSingleWindow<OutputT>> outputFn) {
107124
if (batch.isEmpty()) {
108125
return;
109126
}
110127
LOG.info("Processing batch of {}", batch.size());
111-
List<URI> uris = batch.stream().map(e -> e.uri).collect(Collectors.toList());
112-
remoteFileUtil.download(uris).stream().map(fn::apply).forEach(outputReceiver::output);
113-
if (!keep) {
114-
LOG.info("Deleting batch of {}", batch.size());
115-
remoteFileUtil.delete(uris);
116-
}
117-
batch.clear();
118-
}
128+
List<URI> uris = batch.stream().map(ValueInSingleWindow::getValue).collect(Collectors.toList());
129+
List<Path> paths = remoteFileUtil.download(uris);
119130

120-
private void processBatch(FinishBundleContext c) {
121-
if (batch.isEmpty()) {
122-
return;
123-
}
124-
LOG.info("Processing batch of {}", batch.size());
125-
List<URI> uris = batch.stream().map(e -> e.uri).collect(Collectors.toList());
126-
List<OutputT> outputs =
127-
remoteFileUtil.download(uris).stream().map(fn::apply).collect(Collectors.toList());
128-
// .forEach(c::output);
129-
Iterator<OutputT> i1 = outputs.iterator();
130-
Iterator<Element> i2 = batch.iterator();
131-
while (i1.hasNext() && i2.hasNext()) {
132-
Element e = i2.next();
133-
c.output(i1.next(), e.timestamp, e.window);
131+
Iterator<ValueInSingleWindow<URI>> inputIt = batch.iterator();
132+
Iterator<Path> pathIt = paths.iterator();
133+
while (inputIt.hasNext() && pathIt.hasNext()) {
134+
final ValueInSingleWindow<URI> r = inputIt.next();
135+
final Path path = pathIt.next();
136+
137+
final OutputT o = fn.apply(path);
138+
final Instant ts = r.getTimestamp();
139+
final BoundedWindow w = r.getWindow();
140+
final PaneInfo p = r.getPane();
141+
outputFn.accept(ValueInSingleWindow.of(o, ts, w, p));
134142
}
143+
135144
if (!keep) {
136145
LOG.info("Deleting batch of {}", batch.size());
137146
remoteFileUtil.delete(uris);
138147
}
139148
batch.clear();
140149
}
141-
142-
private class Element {
143-
private URI uri;
144-
private Instant timestamp;
145-
private BoundedWindow window;
146-
147-
Element(URI uri, Instant timestamp, BoundedWindow window) {
148-
this.uri = uri;
149-
this.timestamp = timestamp;
150-
this.window = window;
151-
}
152-
}
153150
}

scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,9 @@
2828
import java.nio.file.Path;
2929
import java.nio.file.Paths;
3030
import java.nio.file.StandardOpenOption;
31+
import java.util.ArrayList;
3132
import java.util.List;
33+
import java.util.Map;
3234
import java.util.concurrent.ExecutionException;
3335
import org.apache.beam.sdk.io.FileSystems;
3436
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
@@ -104,7 +106,12 @@ public Path download(URI src) {
104106
*/
105107
public List<Path> download(List<URI> srcs) {
106108
try {
107-
return paths.getAll(srcs).values().asList();
109+
Map<URI, Path> results = paths.getAll(srcs);
110+
List<Path> paths = new ArrayList<>(srcs.size());
111+
for (URI src : srcs) {
112+
paths.add(results.get(src));
113+
}
114+
return paths;
108115
} catch (ExecutionException e) {
109116
throw new RuntimeException(e);
110117
}

scio-core/src/test/scala/com/spotify/scio/transforms/FileDownloadDoFnTest.scala

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,10 @@
1818
package com.spotify.scio.transforms
1919

2020
import java.nio.file.{Files, Path}
21-
2221
import com.spotify.scio.testing._
2322
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets
2423
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.{Files => GFiles}
24+
import org.joda.time.Instant
2525

2626
import scala.jdk.CollectionConverters._
2727

@@ -31,8 +31,13 @@ class FileDownloadDoFnTest extends PipelineSpec {
3131
val files = createFiles(tmpDir, 100)
3232
runWithContext { sc =>
3333
val p = sc.parallelize(files.map(_.toUri)).flatMapFile(fn)
34-
p.keys should containInAnyOrder((1 to 100).map(_.toString))
35-
p.values.distinct should forAll { f: Path => !Files.exists(f) }
34+
35+
val content = p.keys
36+
val paths = p.values.distinct
37+
38+
val expected = (1 to 100).map(_.toString)
39+
content should containInAnyOrder(expected)
40+
paths should forAll { f: Path => !Files.exists(f) }
3641
}
3742
files.foreach(Files.delete)
3843
Files.delete(tmpDir)
@@ -42,9 +47,23 @@ class FileDownloadDoFnTest extends PipelineSpec {
4247
val tmpDir = Files.createTempDirectory("filedofn-")
4348
val files = createFiles(tmpDir, 100)
4449
runWithContext { sc =>
45-
val p = sc.parallelize(files.map(_.toUri)).flatMapFile(fn, 10, false)
46-
p.keys should containInAnyOrder((1 to 100).map(_.toString))
47-
p.values.distinct should forAll { f: Path => !Files.exists(f) }
50+
// try to use a single bundle so we can check
51+
// elements flushed in processElement as well as
52+
// elements flushed in finishBundle
53+
val p = sc
54+
.parallelize(Seq(files.map(_.toUri).zipWithIndex))
55+
.flatten
56+
.timestampBy { case (_, i) => new Instant(i + 1) }
57+
.keys
58+
.flatMapFile(fn, 10, false)
59+
.withTimestamp
60+
61+
val contentAndTimestamp = p.map { case ((i, _), ts) => (i, ts.getMillis) }
62+
val paths = p.map { case ((_, f), _) => f }.distinct
63+
64+
val expected = (1L to 100L).map(i => (i.toString, i))
65+
contentAndTimestamp should containInAnyOrder(expected)
66+
paths should forAll { f: Path => !Files.exists(f) }
4867
}
4968
files.foreach(Files.delete)
5069
Files.delete(tmpDir)

0 commit comments

Comments
 (0)