-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
DataflowRunner.java
1555 lines (1393 loc) · 64.5 KB
/
DataflowRunner.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.dataflow;
import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Strings.isNullOrEmpty;
import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString;
import com.google.api.client.googleapis.json.GoogleJsonResponseException;
import com.google.api.services.clouddebugger.v2.Clouddebugger;
import com.google.api.services.clouddebugger.v2.model.Debuggee;
import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
import com.google.api.services.dataflow.model.DataflowPackage;
import com.google.api.services.dataflow.model.Job;
import com.google.api.services.dataflow.model.ListJobsResponse;
import com.google.api.services.dataflow.model.WorkerPool;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Strings;
import com.google.common.base.Utf8;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.net.URISyntaxException;
import java.net.URL;
import java.net.URLClassLoader;
import java.nio.channels.Channels;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.beam.runners.core.construction.CoderTranslation;
import org.apache.beam.runners.core.construction.DeduplicatedFlattenFactory;
import org.apache.beam.runners.core.construction.EmptyFlattenAsCreateFactory;
import org.apache.beam.runners.core.construction.PTransformMatchers;
import org.apache.beam.runners.core.construction.PTransformReplacements;
import org.apache.beam.runners.core.construction.ReplacementOutputs;
import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory;
import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource;
import org.apache.beam.runners.core.construction.UnconsumedReads;
import org.apache.beam.runners.core.construction.WriteFilesTranslation;
import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
import org.apache.beam.runners.dataflow.StreamingViewOverrides.StreamingCreatePCollectionViewFactory;
import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions;
import org.apache.beam.runners.dataflow.util.DataflowTemplateJob;
import org.apache.beam.runners.dataflow.util.DataflowTransport;
import org.apache.beam.runners.dataflow.util.MonitoringUtil;
import org.apache.beam.runners.dataflow.util.PropertyNames;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.Pipeline.PipelineVisitor;
import org.apache.beam.sdk.PipelineResult.State;
import org.apache.beam.sdk.PipelineRunner;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.VoidCoder;
import org.apache.beam.sdk.common.runner.v1.RunnerApi;
import org.apache.beam.sdk.extensions.gcp.storage.PathValidator;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.io.WriteFiles;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesCoder;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsValidator;
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.runners.PTransformOverride;
import org.apache.beam.sdk.runners.PTransformOverrideFactory;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.runners.TransformHierarchy.Node;
import org.apache.beam.sdk.state.MapState;
import org.apache.beam.sdk.state.SetState;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.Combine.GroupedValues;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.WithKeys;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.util.InstanceBuilder;
import org.apache.beam.sdk.util.MimeTypes;
import org.apache.beam.sdk.util.NameUtils;
import org.apache.beam.sdk.util.ReleaseInfo;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PCollectionViews;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.PValue;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.ValueWithRecordId;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.joda.time.DateTimeUtils;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A {@link PipelineRunner} that executes the operations in the pipeline by first translating them
* to the Dataflow representation using the {@link DataflowPipelineTranslator} and then submitting
* them to a Dataflow service for execution.
*
* <h3>Permissions</h3>
*
* <p>When reading from a Dataflow source or writing to a Dataflow sink using
* {@code DataflowRunner}, the Google cloudservices account and the Google compute engine service
* account of the GCP project running the Dataflow Job will need access to the corresponding
* source/sink.
*
* <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
* Dataflow Security and Permissions</a> for more details.
*/
public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
private static final Logger LOG = LoggerFactory.getLogger(DataflowRunner.class);
/** Provided configuration options. */
private final DataflowPipelineOptions options;
/** Client for the Dataflow service. This is used to actually submit jobs. */
private final DataflowClient dataflowClient;
/** Translator for this DataflowRunner, based on options. */
private final DataflowPipelineTranslator translator;
/** A set of user defined functions to invoke at different points in execution. */
private DataflowRunnerHooks hooks;
// The limit of CreateJob request size.
private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
@VisibleForTesting
static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024;
private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
/**
* Project IDs must contain lowercase letters, digits, or dashes.
* IDs must start with a letter and may not end with a dash.
* This regex isn't exact - this allows for patterns that would be rejected by
* the service, but this is sufficient for basic validation of project IDs.
*/
public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
/**
* Construct a runner from the provided options.
*
* @param options Properties that configure the runner.
* @return The newly created runner.
*/
public static DataflowRunner fromOptions(PipelineOptions options) {
DataflowPipelineOptions dataflowOptions =
PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
ArrayList<String> missing = new ArrayList<>();
if (dataflowOptions.getAppName() == null) {
missing.add("appName");
}
if (missing.size() > 0) {
throw new IllegalArgumentException(
"Missing required values: " + Joiner.on(',').join(missing));
}
PathValidator validator = dataflowOptions.getPathValidator();
String gcpTempLocation;
try {
gcpTempLocation = dataflowOptions.getGcpTempLocation();
} catch (Exception e) {
throw new IllegalArgumentException("DataflowRunner requires gcpTempLocation, "
+ "but failed to retrieve a value from PipelineOptions", e);
}
validator.validateOutputFilePrefixSupported(gcpTempLocation);
String stagingLocation;
try {
stagingLocation = dataflowOptions.getStagingLocation();
} catch (Exception e) {
throw new IllegalArgumentException("DataflowRunner requires stagingLocation, "
+ "but failed to retrieve a value from PipelineOptions", e);
}
validator.validateOutputFilePrefixSupported(stagingLocation);
if (!Strings.isNullOrEmpty(dataflowOptions.getSaveProfilesToGcs())) {
validator.validateOutputFilePrefixSupported(dataflowOptions.getSaveProfilesToGcs());
}
if (dataflowOptions.getFilesToStage() == null) {
dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
DataflowRunner.class.getClassLoader()));
LOG.info("PipelineOptions.filesToStage was not specified. "
+ "Defaulting to files from the classpath: will stage {} files. "
+ "Enable logging at DEBUG level to see which files will be staged.",
dataflowOptions.getFilesToStage().size());
LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
}
// Verify jobName according to service requirements, truncating converting to lowercase if
// necessary.
String jobName =
dataflowOptions
.getJobName()
.toLowerCase();
checkArgument(
jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
"JobName invalid; the name must consist of only the characters "
+ "[-a-z0-9], starting with a letter and ending with a letter "
+ "or number");
if (!jobName.equals(dataflowOptions.getJobName())) {
LOG.info(
"PipelineOptions.jobName did not match the service requirements. "
+ "Using {} instead of {}.",
jobName,
dataflowOptions.getJobName());
}
dataflowOptions.setJobName(jobName);
// Verify project
String project = dataflowOptions.getProject();
if (project.matches("[0-9]*")) {
throw new IllegalArgumentException("Project ID '" + project
+ "' invalid. Please make sure you specified the Project ID, not project number.");
} else if (!project.matches(PROJECT_ID_REGEXP)) {
throw new IllegalArgumentException("Project ID '" + project
+ "' invalid. Please make sure you specified the Project ID, not project description.");
}
DataflowPipelineDebugOptions debugOptions =
dataflowOptions.as(DataflowPipelineDebugOptions.class);
// Verify the number of worker threads is a valid value
if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
throw new IllegalArgumentException("Number of worker harness threads '"
+ debugOptions.getNumberOfWorkerHarnessThreads()
+ "' invalid. Please make sure the value is non-negative.");
}
if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) {
dataflowOptions.setGcsUploadBufferSizeBytes(GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT);
}
return new DataflowRunner(dataflowOptions);
}
@VisibleForTesting protected DataflowRunner(DataflowPipelineOptions options) {
this.options = options;
this.dataflowClient = DataflowClient.create(options);
this.translator = DataflowPipelineTranslator.fromOptions(options);
this.pcollectionsRequiringIndexedFormat = new HashSet<>();
this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
}
private List<PTransformOverride> getOverrides(boolean streaming) {
ImmutableList.Builder<PTransformOverride> overridesBuilder = ImmutableList.builder();
// Create is implemented in terms of a Read, so it must precede the override to Read in
// streaming
overridesBuilder
.add(
PTransformOverride.of(
PTransformMatchers.flattenWithDuplicateInputs(),
DeduplicatedFlattenFactory.create()))
.add(
PTransformOverride.of(
PTransformMatchers.emptyFlatten(), EmptyFlattenAsCreateFactory.instance()));
if (streaming) {
if (!hasExperiment(options, "enable_custom_pubsub_source")) {
overridesBuilder.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(PubsubUnboundedSource.class),
new ReflectiveRootOverrideFactory(StreamingPubsubIORead.class, this)));
}
if (!hasExperiment(options, "enable_custom_pubsub_sink")) {
overridesBuilder.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(PubsubUnboundedSink.class),
new StreamingPubsubIOWriteOverrideFactory(this)));
}
if (hasExperiment(options, "beam_fn_api")) {
overridesBuilder.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(Create.Values.class),
new StreamingFnApiCreateOverrideFactory()));
}
overridesBuilder
// Support Splittable DoFn for now only in streaming mode.
// The order of the following overrides is important because they are applied in order.
// By default Dataflow runner replaces single-output ParDo with a ParDoSingle override.
// However, we want a different expansion for single-output splittable ParDo.
.add(
PTransformOverride.of(
PTransformMatchers.splittableParDoSingle(),
new ReflectiveOneToOneOverrideFactory(
SplittableParDoOverrides.ParDoSingleViaMulti.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.splittableParDoMulti(),
new SplittableParDoOverrides.SplittableParDoOverrideFactory()))
.add(
PTransformOverride.of(
PTransformMatchers.writeWithRunnerDeterminedSharding(),
new StreamingShardedWriteFactory(options)))
.add(
// Streaming Bounded Read is implemented in terms of Streaming Unbounded Read, and
// must precede it
PTransformOverride.of(
PTransformMatchers.classEqualTo(Read.Bounded.class),
new ReflectiveRootOverrideFactory(StreamingBoundedRead.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(Read.Unbounded.class),
new ReflectiveRootOverrideFactory(StreamingUnboundedRead.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(View.CreatePCollectionView.class),
new StreamingCreatePCollectionViewFactory()));
} else {
overridesBuilder
// State and timer pardos are implemented by expansion to GBK-then-ParDo
.add(
PTransformOverride.of(
PTransformMatchers.stateOrTimerParDoMulti(),
BatchStatefulParDoOverrides.multiOutputOverrideFactory()))
.add(
PTransformOverride.of(
PTransformMatchers.stateOrTimerParDoSingle(),
BatchStatefulParDoOverrides.singleOutputOverrideFactory()))
.add(
PTransformOverride.of(
PTransformMatchers.createViewWithViewFn(PCollectionViews.MapViewFn.class),
new ReflectiveOneToOneOverrideFactory(
BatchViewOverrides.BatchViewAsMap.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.createViewWithViewFn(PCollectionViews.MultimapViewFn.class),
new ReflectiveOneToOneOverrideFactory(
BatchViewOverrides.BatchViewAsMultimap.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.createViewWithViewFn(PCollectionViews.SingletonViewFn.class),
new ReflectiveOneToOneOverrideFactory(
BatchViewOverrides.BatchViewAsSingleton.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.createViewWithViewFn(PCollectionViews.ListViewFn.class),
new ReflectiveOneToOneOverrideFactory(
BatchViewOverrides.BatchViewAsList.class, this)))
.add(
PTransformOverride.of(
PTransformMatchers.createViewWithViewFn(PCollectionViews.IterableViewFn.class),
new ReflectiveOneToOneOverrideFactory(
BatchViewOverrides.BatchViewAsIterable.class, this)));
}
overridesBuilder
.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(Reshuffle.class), new ReshuffleOverrideFactory()))
// Order is important. Streaming views almost all use Combine internally.
.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(Combine.GroupedValues.class),
new PrimitiveCombineGroupedValuesOverrideFactory()))
.add(
PTransformOverride.of(
PTransformMatchers.classEqualTo(ParDo.SingleOutput.class),
new PrimitiveParDoSingleFactory()));
return overridesBuilder.build();
}
private static class ReflectiveOneToOneOverrideFactory<
InputT, OutputT, TransformT extends PTransform<PCollection<InputT>, PCollection<OutputT>>>
extends SingleInputOutputOverrideFactory<
PCollection<InputT>, PCollection<OutputT>, TransformT> {
private final Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement;
private final DataflowRunner runner;
private ReflectiveOneToOneOverrideFactory(
Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement,
DataflowRunner runner) {
this.replacement = replacement;
this.runner = runner;
}
@Override
public PTransformReplacement<PCollection<InputT>, PCollection<OutputT>> getReplacementTransform(
AppliedPTransform<PCollection<InputT>, PCollection<OutputT>, TransformT> transform) {
PTransform<PCollection<InputT>, PCollection<OutputT>> rep =
InstanceBuilder.ofType(replacement)
.withArg(DataflowRunner.class, runner)
.withArg(
(Class<TransformT>) transform.getTransform().getClass(), transform.getTransform())
.build();
return PTransformReplacement.of(PTransformReplacements.getSingletonMainInput(transform), rep);
}
}
private static class ReflectiveRootOverrideFactory<T>
implements PTransformOverrideFactory<
PBegin, PCollection<T>, PTransform<PInput, PCollection<T>>> {
private final Class<PTransform<PBegin, PCollection<T>>> replacement;
private final DataflowRunner runner;
private ReflectiveRootOverrideFactory(
Class<PTransform<PBegin, PCollection<T>>> replacement, DataflowRunner runner) {
this.replacement = replacement;
this.runner = runner;
}
@Override
public PTransformReplacement<PBegin, PCollection<T>> getReplacementTransform(
AppliedPTransform<PBegin, PCollection<T>, PTransform<PInput, PCollection<T>>> transform) {
PTransform<PInput, PCollection<T>> original = transform.getTransform();
return PTransformReplacement.of(
transform.getPipeline().begin(),
InstanceBuilder.ofType(replacement)
.withArg(DataflowRunner.class, runner)
.withArg(
(Class<? super PTransform<PInput, PCollection<T>>>) original.getClass(), original)
.build());
}
@Override
public Map<PValue, ReplacementOutput> mapOutputs(
Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
return ReplacementOutputs.singleton(outputs, newOutput);
}
}
private String debuggerMessage(String projectId, String uniquifier) {
return String.format("To debug your job, visit Google Cloud Debugger at: "
+ "https://console.developers.google.com/debug?project=%s&dbgee=%s",
projectId, uniquifier);
}
private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
if (!options.getEnableCloudDebugger()) {
return;
}
if (options.getDebuggee() != null) {
throw new RuntimeException("Should not specify the debuggee");
}
Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
options.setDebuggee(debuggee);
System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
}
private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
registerReq.setDebuggee(new Debuggee()
.setProject(options.getProject())
.setUniquifier(uniquifier)
.setDescription(uniquifier)
.setAgentVersion("google.com/cloud-dataflow-java/v1"));
try {
RegisterDebuggeeResponse registerResponse =
debuggerClient.controller().debuggees().register(registerReq).execute();
Debuggee debuggee = registerResponse.getDebuggee();
if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
throw new RuntimeException("Unable to register with the debugger: "
+ debuggee.getStatus().getDescription().getFormat());
}
return debuggee;
} catch (IOException e) {
throw new RuntimeException("Unable to register with the debugger: ", e);
}
}
@Override
public DataflowPipelineJob run(Pipeline pipeline) {
logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
if (containsUnboundedPCollection(pipeline)) {
options.setStreaming(true);
}
replaceTransforms(pipeline);
LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
+ "related to Google Compute Engine usage and other Google Cloud Services.");
List<DataflowPackage> packages = options.getStager().stageFiles();
// Set a unique client_request_id in the CreateJob request.
// This is used to ensure idempotence of job creation across retried
// attempts to create a job. Specifically, if the service returns a job with
// a different client_request_id, it means the returned one is a different
// job previously created with the same job name, and that the job creation
// has been effectively rejected. The SDK should return
// Error::Already_Exists to user in that case.
int randomNum = new Random().nextInt(9000) + 1000;
String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
.print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
// Try to create a debuggee ID. This must happen before the job is translated since it may
// update the options.
DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
maybeRegisterDebuggee(dataflowOptions, requestId);
JobSpecification jobSpecification =
translator.translate(pipeline, this, packages);
Job newJob = jobSpecification.getJob();
newJob.setClientRequestId(requestId);
ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo();
String version = releaseInfo.getVersion();
checkState(
!version.equals("${pom.version}"),
"Unable to submit a job to the Dataflow service with unset version ${pom.version}");
System.out.println("Dataflow SDK version: " + version);
newJob.getEnvironment().setUserAgent((Map) releaseInfo.getProperties());
// The Dataflow Service may write to the temporary directory directly, so
// must be verified.
if (!isNullOrEmpty(options.getGcpTempLocation())) {
newJob.getEnvironment().setTempStoragePrefix(
dataflowOptions.getPathValidator().verifyPath(options.getGcpTempLocation()));
}
newJob.getEnvironment().setDataset(options.getTempDatasetId());
newJob.getEnvironment().setExperiments(options.getExperiments());
// Set the Docker container image that executes Dataflow worker harness, residing in Google
// Container Registry. Translator is guaranteed to create a worker pool prior to this point.
String workerHarnessContainerImage = getContainerImageForJob(options);
for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
}
newJob.getEnvironment().setVersion(getEnvironmentVersion(options));
if (hooks != null) {
hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
}
if (!isNullOrEmpty(options.getDataflowJobFile())
|| !isNullOrEmpty(options.getTemplateLocation())) {
boolean isTemplate = !isNullOrEmpty(options.getTemplateLocation());
if (isTemplate) {
checkArgument(isNullOrEmpty(options.getDataflowJobFile()),
"--dataflowJobFile and --templateLocation are mutually exclusive.");
}
String fileLocation = firstNonNull(
options.getTemplateLocation(), options.getDataflowJobFile());
checkArgument(
fileLocation.startsWith("/") || fileLocation.startsWith("gs://"),
"Location must be local or on Cloud Storage, got %s.",
fileLocation);
ResourceId fileResource = FileSystems.matchNewResource(fileLocation, false /* isDirectory */);
String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
try (PrintWriter printWriter =
new PrintWriter(
Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)))) {
printWriter.print(workSpecJson);
LOG.info("Printed job specification to {}", fileLocation);
} catch (IOException ex) {
String error =
String.format("Cannot create output file at %s", fileLocation);
if (isTemplate) {
throw new RuntimeException(error, ex);
} else {
LOG.warn(error, ex);
}
}
if (isTemplate) {
LOG.info("Template successfully created.");
return new DataflowTemplateJob();
}
}
String jobIdToUpdate = null;
if (options.isUpdate()) {
jobIdToUpdate = getJobIdFromName(options.getJobName());
newJob.setTransformNameMapping(options.getTransformNameMapping());
newJob.setReplaceJobId(jobIdToUpdate);
}
Job jobResult;
try {
jobResult = dataflowClient.createJob(newJob);
} catch (GoogleJsonResponseException e) {
String errorMessages = "Unexpected errors";
if (e.getDetails() != null) {
if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
errorMessages = "The size of the serialized JSON representation of the pipeline "
+ "exceeds the allowable limit. "
+ "For more information, please check the FAQ link below:\n"
+ "https://cloud.google.com/dataflow/faq";
} else {
errorMessages = e.getDetails().getMessage();
}
}
throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
} catch (IOException e) {
throw new RuntimeException("Failed to create a workflow job", e);
}
// Use a raw client for post-launch monitoring, as status calls may fail
// regularly and need not be retried automatically.
DataflowPipelineJob dataflowPipelineJob =
new DataflowPipelineJob(
DataflowClient.create(options),
jobResult.getId(),
options,
jobSpecification.getStepNames());
// If the service returned client request id, the SDK needs to compare it
// with the original id generated in the request, if they are not the same
// (i.e., the returned job is not created by this request), throw
// DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedException
// depending on whether this is a reload or not.
if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
&& !jobResult.getClientRequestId().equals(requestId)) {
// If updating a job.
if (options.isUpdate()) {
throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
String.format("The job named %s with id: %s has already been updated into job id: %s "
+ "and cannot be updated again.",
newJob.getName(), jobIdToUpdate, jobResult.getId()));
} else {
throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
String.format("There is already an active job named %s with id: %s. If you want "
+ "to submit a second job, try again by setting a different name using --jobName.",
newJob.getName(), jobResult.getId()));
}
}
LOG.info("To access the Dataflow monitoring console, please navigate to {}",
MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
System.out.println("Submitted job: " + jobResult.getId());
LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
return dataflowPipelineJob;
}
/** Returns true if the specified experiment is enabled, handling null experiments. */
public static boolean hasExperiment(DataflowPipelineDebugOptions options, String experiment) {
List<String> experiments =
firstNonNull(options.getExperiments(), Collections.<String>emptyList());
return experiments.contains(experiment);
}
/** Helper to configure the Dataflow Job Environment based on the user's job options. */
private static Map<String, Object> getEnvironmentVersion(DataflowPipelineOptions options) {
DataflowRunnerInfo runnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo();
String majorVersion;
String jobType;
if (hasExperiment(options, "beam_fn_api")) {
majorVersion = runnerInfo.getFnApiEnvironmentMajorVersion();
jobType = options.isStreaming() ? "FNAPI_STREAMING" : "FNAPI_BATCH";
} else {
majorVersion = runnerInfo.getLegacyEnvironmentMajorVersion();
jobType = options.isStreaming() ? "STREAMING" : "JAVA_BATCH_AUTOSCALING";
}
return ImmutableMap.<String, Object>of(
PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, majorVersion,
PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
}
@VisibleForTesting
void replaceTransforms(Pipeline pipeline) {
boolean streaming = options.isStreaming() || containsUnboundedPCollection(pipeline);
// Ensure all outputs of all reads are consumed before potentially replacing any
// Read PTransforms
UnconsumedReads.ensureAllReadsConsumed(pipeline);
pipeline.replaceAll(getOverrides(streaming));
}
private boolean containsUnboundedPCollection(Pipeline p) {
class BoundednessVisitor extends PipelineVisitor.Defaults {
IsBounded boundedness = IsBounded.BOUNDED;
@Override
public void visitValue(PValue value, Node producer) {
if (value instanceof PCollection) {
boundedness = boundedness.and(((PCollection) value).isBounded());
}
}
}
BoundednessVisitor visitor = new BoundednessVisitor();
p.traverseTopologically(visitor);
return visitor.boundedness == IsBounded.UNBOUNDED;
};
/**
* Returns the DataflowPipelineTranslator associated with this object.
*/
public DataflowPipelineTranslator getTranslator() {
return translator;
}
/**
* Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}.
*/
@Experimental
public void setHooks(DataflowRunnerHooks hooks) {
this.hooks = hooks;
}
/////////////////////////////////////////////////////////////////////////////
/** Outputs a warning about PCollection views without deterministic key coders. */
private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
// We need to wait till this point to determine the names of the transforms since only
// at this time do we know the hierarchy of the transforms otherwise we could
// have just recorded the full names during apply time.
if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
pipeline.traverseTopologically(
new PipelineVisitor.Defaults() {
@Override
public void visitValue(PValue value, TransformHierarchy.Node producer) {}
@Override
public void visitPrimitiveTransform(TransformHierarchy.Node node) {
if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
}
}
@Override
public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
if (node.getTransform() instanceof View.AsMap
|| node.getTransform() instanceof View.AsMultimap) {
PCollection<KV<?, ?>> input =
(PCollection<KV<?, ?>>) Iterables.getOnlyElement(node.getInputs().values());
KvCoder<?, ?> inputCoder = (KvCoder) input.getCoder();
try {
inputCoder.getKeyCoder().verifyDeterministic();
} catch (NonDeterministicException e) {
ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
}
}
if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
}
return CompositeBehavior.ENTER_TRANSFORM;
}
@Override
public void leaveCompositeTransform(TransformHierarchy.Node node) {}
});
LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
+ "because the key coder is not deterministic. Falling back to singleton implementation "
+ "which may cause memory and/or performance problems. Future major versions of "
+ "Dataflow will require deterministic key coders.",
ptransformViewNamesWithNonDeterministicKeyCoders);
}
}
/**
* Returns true if the passed in {@link PCollection} needs to be materialiazed using
* an indexed format.
*/
boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
return pcollectionsRequiringIndexedFormat.contains(pcol);
}
/**
* Marks the passed in {@link PCollection} as requiring to be materialized using
* an indexed format.
*/
void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
pcollectionsRequiringIndexedFormat.add(pcol);
}
/** A set of {@link View}s with non-deterministic key coders. */
private Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
/**
* Records that the {@link PTransform} requires a deterministic key coder.
*/
void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
}
// ================================================================================
// PubsubIO translations
// ================================================================================
/**
* Suppress application of {@link PubsubUnboundedSource#expand} in streaming mode so that we can
* instead defer to Windmill's implementation.
*/
private static class StreamingPubsubIORead
extends PTransform<PBegin, PCollection<PubsubMessage>> {
private final PubsubUnboundedSource transform;
/** Builds an instance of this class from the overridden transform. */
@SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
public StreamingPubsubIORead(DataflowRunner runner, PubsubUnboundedSource transform) {
this.transform = transform;
}
PubsubUnboundedSource getOverriddenTransform() {
return transform;
}
@Override
public PCollection<PubsubMessage> expand(PBegin input) {
return PCollection.<PubsubMessage>createPrimitiveOutputInternal(
input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED)
.setCoder(new PubsubMessageWithAttributesCoder());
}
@Override
protected String getKindString() {
return "StreamingPubsubIORead";
}
static {
DataflowPipelineTranslator.registerTransformTranslator(
StreamingPubsubIORead.class, new StreamingPubsubIOReadTranslator());
}
}
/** Rewrite {@link StreamingPubsubIORead} to the appropriate internal node. */
private static class StreamingPubsubIOReadTranslator
implements TransformTranslator<StreamingPubsubIORead> {
@Override
public void translate(StreamingPubsubIORead transform, TranslationContext context) {
checkArgument(
context.getPipelineOptions().isStreaming(),
"StreamingPubsubIORead is only for streaming pipelines.");
PubsubUnboundedSource overriddenTransform = transform.getOverriddenTransform();
StepTranslationContext stepContext = context.addStep(transform, "ParallelRead");
stepContext.addInput(PropertyNames.FORMAT, "pubsub");
if (overriddenTransform.getTopicProvider() != null) {
if (overriddenTransform.getTopicProvider().isAccessible()) {
stepContext.addInput(
PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path());
} else {
stepContext.addInput(
PropertyNames.PUBSUB_TOPIC_OVERRIDE,
((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName());
}
}
if (overriddenTransform.getSubscriptionProvider() != null) {
if (overriddenTransform.getSubscriptionProvider().isAccessible()) {
stepContext.addInput(
PropertyNames.PUBSUB_SUBSCRIPTION,
overriddenTransform.getSubscription().getV1Beta1Path());
} else {
stepContext.addInput(
PropertyNames.PUBSUB_SUBSCRIPTION_OVERRIDE,
((NestedValueProvider) overriddenTransform.getSubscriptionProvider()).propertyName());
}
}
if (overriddenTransform.getTimestampAttribute() != null) {
stepContext.addInput(
PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, overriddenTransform.getTimestampAttribute());
}
if (overriddenTransform.getIdAttribute() != null) {
stepContext.addInput(
PropertyNames.PUBSUB_ID_ATTRIBUTE, overriddenTransform.getIdAttribute());
}
// In both cases, the transform needs to read PubsubMessage. However, in case it needs
// the attributes, we supply an identity "parse fn" so the worker will read PubsubMessage's
// from Windmill and simply pass them around; and in case it doesn't need attributes,
// we're already implicitly using a "Coder" that interprets the data as a PubsubMessage's
// payload.
if (overriddenTransform.getNeedsAttributes()) {
stepContext.addInput(
PropertyNames.PUBSUB_SERIALIZED_ATTRIBUTES_FN,
byteArrayToJsonString(
serializeToByteArray(new IdentityMessageFn())));
}
stepContext.addOutput(context.getOutput(transform));
}
}
private static class IdentityMessageFn
extends SimpleFunction<PubsubMessage, PubsubMessage> {
@Override
public PubsubMessage apply(PubsubMessage input) {
return input;
}
}
/**
* Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode so that we can
* instead defer to Windmill's implementation.
*/
private static class StreamingPubsubIOWrite
extends PTransform<PCollection<PubsubMessage>, PDone> {
private final PubsubUnboundedSink transform;
/**
* Builds an instance of this class from the overridden transform.
*/
public StreamingPubsubIOWrite(
DataflowRunner runner, PubsubUnboundedSink transform) {
this.transform = transform;
}
PubsubUnboundedSink getOverriddenTransform() {
return transform;
}
@Override
public PDone expand(PCollection<PubsubMessage> input) {
return PDone.in(input.getPipeline());
}
@Override
protected String getKindString() {
return "StreamingPubsubIOWrite";
}
static {
DataflowPipelineTranslator.registerTransformTranslator(
StreamingPubsubIOWrite.class, new StreamingPubsubIOWriteTranslator());
}
}
/**
* Rewrite {@link StreamingPubsubIOWrite} to the appropriate internal node.
*/
private static class StreamingPubsubIOWriteTranslator implements
TransformTranslator<StreamingPubsubIOWrite> {
@Override
public void translate(
StreamingPubsubIOWrite transform,
TranslationContext context) {
checkArgument(context.getPipelineOptions().isStreaming(),
"StreamingPubsubIOWrite is only for streaming pipelines.");
PubsubUnboundedSink overriddenTransform = transform.getOverriddenTransform();
StepTranslationContext stepContext = context.addStep(transform, "ParallelWrite");
stepContext.addInput(PropertyNames.FORMAT, "pubsub");
if (overriddenTransform.getTopicProvider().isAccessible()) {
stepContext.addInput(
PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path());
} else {
stepContext.addInput(
PropertyNames.PUBSUB_TOPIC_OVERRIDE,
((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName());
}
if (overriddenTransform.getTimestampAttribute() != null) {
stepContext.addInput(
PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, overriddenTransform.getTimestampAttribute());
}
if (overriddenTransform.getIdAttribute() != null) {
stepContext.addInput(