1717 */
1818package $ {package }.complete .game .utils ;
1919
20- import static com .google .common .base .Preconditions .checkArgument ;
21- import static com .google .common .base .Verify .verifyNotNull ;
22-
23- import java .io .Serializable ;
24- import java .util .ArrayList ;
25- import java .util .List ;
26- import java .util .Map ;
27- import java .util .TimeZone ;
28- import java .util .stream .Collectors ;
29- import org .apache .beam .sdk .io .FileBasedSink ;
30- import org .apache .beam .sdk .io .FileBasedSink .FilenamePolicy ;
31- import org .apache .beam .sdk .io .TextIO ;
32- import org .apache .beam .sdk .io .fs .ResolveOptions .StandardResolveOptions ;
33- import org .apache .beam .sdk .io .fs .ResourceId ;
34- import org .apache .beam .sdk .transforms .DoFn ;
35- import org .apache .beam .sdk .transforms .PTransform ;
36- import org .apache .beam .sdk .transforms .ParDo ;
37- import org .apache .beam .sdk .transforms .windowing .BoundedWindow ;
38- import org .apache .beam .sdk .transforms .windowing .IntervalWindow ;
39- import org .apache .beam .sdk .values .PCollection ;
40- import org .apache .beam .sdk .values .PDone ;
41- import org .joda .time .DateTimeZone ;
42- import org .joda .time .format .DateTimeFormat ;
43- import org .joda .time .format .DateTimeFormatter ;
20+ import static com .google .common .base .Preconditions .checkArgument ;
21+
22+ import java .io .Serializable ;
23+ import java .util .ArrayList ;
24+ import java .util .List ;
25+ import java .util .Map ;
26+ import java .util .TimeZone ;
27+ import java .util .stream .Collectors ;
28+ import org .apache .beam .sdk .io .FileBasedSink ;
29+ import org .apache .beam .sdk .io .FileBasedSink .FilenamePolicy ;
30+ import org .apache .beam .sdk .io .FileBasedSink .OutputFileHints ;
31+ import org .apache .beam .sdk .io .TextIO ;
32+ import org .apache .beam .sdk .io .fs .ResolveOptions .StandardResolveOptions ;
33+ import org .apache .beam .sdk .io .fs .ResourceId ;
34+ import org .apache .beam .sdk .transforms .DoFn ;
35+ import org .apache .beam .sdk .transforms .PTransform ;
36+ import org .apache .beam .sdk .transforms .ParDo ;
37+ import org .apache .beam .sdk .transforms .windowing .BoundedWindow ;
38+ import org .apache .beam .sdk .transforms .windowing .IntervalWindow ;
39+ import org .apache .beam .sdk .transforms .windowing .PaneInfo ;
40+ import org .apache .beam .sdk .values .PCollection ;
41+ import org .apache .beam .sdk .values .PDone ;
42+ import org .joda .time .DateTimeZone ;
43+ import org .joda .time .format .DateTimeFormat ;
44+ import org .joda .time .format .DateTimeFormatter ;
4445
4546/**
4647 * Generate, format, and write rows. Use provided information about the field names and types, as
@@ -111,21 +112,12 @@ public PDone expand(PCollection<String> input) {
111112 checkArgument (
112113 input .getWindowingStrategy ().getWindowFn ().windowCoder () == IntervalWindow .getCoder ());
113114
114- // filenamePrefix may contain a directory and a filename component. Pull out only the filename
115- // component from that path for the PerWindowFiles.
116- String prefix = "" ;
117115 ResourceId resource = FileBasedSink .convertToFileResourceIfPossible (filenamePrefix );
118- if (!resource .isDirectory ()) {
119- prefix = verifyNotNull (
120- resource .getFilename (),
121- "A non-directory resource should have a non-null filename: %s" ,
122- resource );
123- }
124116
125117 return input .apply (
126118 TextIO .write ()
127- .to (resource . getCurrentDirectory ( ))
128- .withFilenamePolicy ( new PerWindowFiles ( prefix ))
119+ .to (new PerWindowFiles ( resource ))
120+ .withTempDirectory ( resource . getCurrentDirectory ( ))
129121 .withWindowedWrites ()
130122 .withNumShards (3 ));
131123 }
@@ -139,31 +131,38 @@ public PDone expand(PCollection<String> input) {
139131 */
140132 protected static class PerWindowFiles extends FilenamePolicy {
141133
142- private final String prefix ;
134+ private final ResourceId prefix ;
143135
144- public PerWindowFiles (String prefix ) {
136+ public PerWindowFiles (ResourceId prefix ) {
145137 this .prefix = prefix ;
146138 }
147139
148140 public String filenamePrefixForWindow (IntervalWindow window ) {
149- return String .format ("%s-%s-%s" ,
150- prefix , formatter .print (window .start ()), formatter .print (window .end ()));
141+ String filePrefix = prefix .isDirectory () ? "" : prefix .getFilename ();
142+ return String .format (
143+ "%s-%s-%s" , filePrefix , formatter .print (window .start ()), formatter .print (window .end ()));
151144 }
152145
153146 @ Override
154- public ResourceId windowedFilename (
155- ResourceId outputDirectory , WindowedContext context , String extension ) {
156- IntervalWindow window = (IntervalWindow ) context .getWindow ();
157- String filename = String .format (
158- "%s-%s-of-%s%s" ,
159- filenamePrefixForWindow (window ), context .getShardNumber (), context .getNumShards (),
160- extension );
161- return outputDirectory .resolve (filename , StandardResolveOptions .RESOLVE_FILE );
147+ public ResourceId windowedFilename (int shardNumber ,
148+ int numShards ,
149+ BoundedWindow window ,
150+ PaneInfo paneInfo ,
151+ OutputFileHints outputFileHints ) {
152+ IntervalWindow intervalWindow = (IntervalWindow ) window ;
153+ String filename =
154+ String .format (
155+ "%s-%s-of-%s%s" ,
156+ filenamePrefixForWindow (intervalWindow ),
157+ shardNumber ,
158+ numShards ,
159+ outputFileHints .getSuggestedFilenameSuffix ());
160+ return prefix .getCurrentDirectory ().resolve (filename , StandardResolveOptions .RESOLVE_FILE );
162161 }
163162
164163 @ Override
165164 public ResourceId unwindowedFilename (
166- ResourceId outputDirectory , Context context , String extension ) {
165+ int shardNumber , int numShards , OutputFileHints outputFileHints ) {
167166 throw new UnsupportedOperationException ("Unsupported." );
168167 }
169168 }
0 commit comments