20
20
import com .spotify .scio .util .RemoteFileUtil ;
21
21
import java .net .URI ;
22
22
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 ;
26
25
import java .util .stream .Collectors ;
27
26
import org .apache .beam .sdk .transforms .DoFn ;
28
27
import org .apache .beam .sdk .transforms .SerializableFunction ;
29
28
import org .apache .beam .sdk .transforms .display .DisplayData ;
30
29
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 ;
31
32
import org .joda .time .Instant ;
32
33
import org .slf4j .Logger ;
33
34
import org .slf4j .LoggerFactory ;
@@ -37,7 +38,7 @@ public class FileDownloadDoFn<OutputT> extends DoFn<URI, OutputT> {
37
38
38
39
private static final Logger LOG = LoggerFactory .getLogger (FileDownloadDoFn .class );
39
40
40
- private final List <Element > batch ;
41
+ private final List <ValueInSingleWindow < URI > > batch ;
41
42
private final RemoteFileUtil remoteFileUtil ;
42
43
private final SerializableFunction <Path , OutputT > fn ;
43
44
private final int batchSize ;
@@ -78,21 +79,37 @@ public void startBundle(StartBundleContext context) {
78
79
this .batch .clear ();
79
80
}
80
81
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
+
81
90
@ ProcessElement
82
91
public void processElement (
83
92
@ DoFn .Element URI element ,
84
93
@ 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 ));
88
98
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
+ });
90
107
}
91
108
}
92
109
93
110
@ FinishBundle
94
111
public void finishBundle (FinishBundleContext context ) {
95
- processBatch ( context );
112
+ flush ( p -> context . output ( p . getValue (), p . getTimestamp (), p . getWindow ()) );
96
113
}
97
114
98
115
@ Override
@@ -103,51 +120,31 @@ public void populateDisplayData(DisplayData.Builder builder) {
103
120
.add (DisplayData .item ("Keep Downloaded Files" , keep ));
104
121
}
105
122
106
- private void processBatch ( OutputReceiver < OutputT > outputReceiver ) {
123
+ private void flush ( Consumer < ValueInSingleWindow < OutputT >> outputFn ) {
107
124
if (batch .isEmpty ()) {
108
125
return ;
109
126
}
110
127
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 );
119
130
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 ));
134
142
}
143
+
135
144
if (!keep ) {
136
145
LOG .info ("Deleting batch of {}" , batch .size ());
137
146
remoteFileUtil .delete (uris );
138
147
}
139
148
batch .clear ();
140
149
}
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
- }
153
150
}
0 commit comments