/
StreamingDataflowWorker.java
2471 lines (2230 loc) · 98.1 KB
/
StreamingDataflowWorker.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.worker;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.beam.runners.dataflow.DataflowRunner.hasExperiment;
import static org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.THROTTLING_MSECS_METRIC_NAME;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import com.google.api.services.dataflow.model.CounterStructuredName;
import com.google.api.services.dataflow.model.CounterUpdate;
import com.google.api.services.dataflow.model.MapTask;
import com.google.api.services.dataflow.model.Status;
import com.google.api.services.dataflow.model.StreamingComputationConfig;
import com.google.api.services.dataflow.model.StreamingConfigTask;
import com.google.api.services.dataflow.model.WorkItem;
import com.google.api.services.dataflow.model.WorkItemStatus;
import com.google.auto.value.AutoValue;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Deque;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.Semaphore;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.Supplier;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.beam.runners.core.metrics.ExecutionStateSampler;
import org.apache.beam.runners.core.metrics.ExecutionStateTracker;
import org.apache.beam.runners.core.metrics.MetricsLogger;
import org.apache.beam.runners.dataflow.DataflowRunner;
import org.apache.beam.runners.dataflow.internal.CustomSources;
import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
import org.apache.beam.runners.dataflow.util.CloudObject;
import org.apache.beam.runners.dataflow.util.CloudObjects;
import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingPerStageSystemCounterNames;
import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingSystemCounterNames;
import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.Work.State;
import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionStateRegistry;
import org.apache.beam.runners.dataflow.worker.apiary.FixMultiOutputInfosOnParDoInstructions;
import org.apache.beam.runners.dataflow.worker.counters.Counter;
import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
import org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
import org.apache.beam.runners.dataflow.worker.counters.NameContext;
import org.apache.beam.runners.dataflow.worker.graph.Edges.Edge;
import org.apache.beam.runners.dataflow.worker.graph.MapTaskToNetworkFunction;
import org.apache.beam.runners.dataflow.worker.graph.Networks;
import org.apache.beam.runners.dataflow.worker.graph.Nodes.InstructionOutputNode;
import org.apache.beam.runners.dataflow.worker.graph.Nodes.Node;
import org.apache.beam.runners.dataflow.worker.graph.Nodes.ParallelInstructionNode;
import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions;
import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler;
import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet;
import org.apache.beam.runners.dataflow.worker.status.DebugCapture;
import org.apache.beam.runners.dataflow.worker.status.DebugCapture.Capturable;
import org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider;
import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider;
import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages;
import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter;
import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter;
import org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.CommitWorkStream;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.GetWorkStream;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.StreamPool;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.extensions.gcp.util.Transport;
import org.apache.beam.sdk.fn.IdGenerator;
import org.apache.beam.sdk.fn.IdGenerators;
import org.apache.beam.sdk.fn.JvmInitializers;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.metrics.MetricName;
import org.apache.beam.sdk.metrics.MetricsEnvironment;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.sdk.util.BackOffUtils;
import org.apache.beam.sdk.util.FluentBackoff;
import org.apache.beam.sdk.util.Sleeper;
import org.apache.beam.sdk.util.UserCodeException;
import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.EvictingQueue;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ListMultimap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.MultimapBuilder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.graph.MutableNetwork;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.net.HostAndPort;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Implements a Streaming Dataflow worker. */
@SuppressWarnings({
"rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class StreamingDataflowWorker {
private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class);
/** The idGenerator to generate unique id globally. */
private static final IdGenerator idGenerator = IdGenerators.decrementingLongs();
/**
* Fix up MapTask representation because MultiOutputInfos are missing from system generated
* ParDoInstructions.
*/
private static final Function<MapTask, MapTask> fixMultiOutputInfos =
new FixMultiOutputInfosOnParDoInstructions(idGenerator);
/**
* Function which converts map tasks to their network representation for execution.
*
* <ul>
* <li>Translate the map task to a network representation.
* <li>Remove flatten instructions by rewiring edges.
* </ul>
*/
private static final Function<MapTask, MutableNetwork<Node, Edge>> mapTaskToBaseNetwork =
new MapTaskToNetworkFunction(idGenerator);
private static Random clientIdGenerator = new Random();
// Maximum number of threads for processing. Currently each thread processes one key at a time.
static final int MAX_PROCESSING_THREADS = 300;
static final long THREAD_EXPIRATION_TIME_SEC = 60;
static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20;
static final int MAX_COMMIT_QUEUE_BYTES = 500 << 20; // 500MB
static final int NUM_COMMIT_STREAMS = 1;
static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3;
static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1);
private static final int DEFAULT_STATUS_PORT = 8081;
// Maximum size of the result of a GetWork request.
private static final long MAX_GET_WORK_FETCH_BYTES = 64L << 20; // 64m
// Reserved ID for counter updates.
// Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
/** Maximum number of failure stacktraces to report in each update sent to backend. */
private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000;
// TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic
// throttling-msecs metric.
public static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME =
MetricName.named(
"org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl",
"throttling-msecs");
private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION = Duration.standardMinutes(5);
/** Returns whether an exception was caused by a {@link OutOfMemoryError}. */
private static boolean isOutOfMemoryError(Throwable t) {
while (t != null) {
if (t instanceof OutOfMemoryError) {
return true;
}
t = t.getCause();
}
return false;
}
private static class KeyCommitTooLargeException extends Exception {
public static KeyCommitTooLargeException causedBy(
String computationId, long byteLimit, WorkItemCommitRequest request) {
StringBuilder message = new StringBuilder();
message.append("Commit request for stage ");
message.append(computationId);
message.append(" and key ");
message.append(request.getKey().toStringUtf8());
if (request.getSerializedSize() > 0) {
message.append(
" has size "
+ request.getSerializedSize()
+ " which is more than the limit of "
+ byteLimit);
} else {
message.append(" is larger than 2GB and cannot be processed");
}
message.append(
". This may be caused by grouping a very "
+ "large amount of data in a single window without using Combine,"
+ " or by producing a large amount of data from a single input element.");
return new KeyCommitTooLargeException(message.toString());
}
private KeyCommitTooLargeException(String message) {
super(message);
}
}
private static MapTask parseMapTask(String input) throws IOException {
return Transport.getJsonFactory().fromString(input, MapTask.class);
}
public static void main(String[] args) throws Exception {
JvmInitializers.runOnStartup();
DataflowWorkerHarnessHelper.initializeLogging(StreamingDataflowWorker.class);
DataflowWorkerHarnessOptions options =
DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions(
StreamingDataflowWorker.class);
DataflowWorkerHarnessHelper.configureLogging(options);
checkArgument(
options.isStreaming(),
"%s instantiated with options indicating batch use",
StreamingDataflowWorker.class.getName());
checkArgument(
!DataflowRunner.hasExperiment(options, "beam_fn_api"),
"%s cannot be main() class with beam_fn_api enabled",
StreamingDataflowWorker.class.getSimpleName());
StreamingDataflowWorker worker =
StreamingDataflowWorker.fromDataflowWorkerHarnessOptions(options);
// Use the MetricsLogger container which is used by BigQueryIO to periodically log process-wide
// metrics.
MetricsEnvironment.setProcessWideContainer(new MetricsLogger(null));
JvmInitializers.runBeforeProcessing(options);
worker.startStatusPages();
worker.start();
}
/** Bounded set of queues, with a maximum total weight. */
private static class WeightedBoundedQueue<V> {
private final LinkedBlockingQueue<V> queue = new LinkedBlockingQueue<>();
private final int maxWeight;
private final Semaphore limit;
private final Function<V, Integer> weigher;
public WeightedBoundedQueue(int maxWeight, Function<V, Integer> weigher) {
this.maxWeight = maxWeight;
this.limit = new Semaphore(maxWeight, true);
this.weigher = weigher;
}
/**
* Adds the value to the queue, blocking if this would cause the overall weight to exceed the
* limit.
*/
public void put(V value) {
limit.acquireUninterruptibly(weigher.apply(value));
queue.add(value);
}
/** Returns and removes the next value, or null if there is no such value. */
public @Nullable V poll() {
V result = queue.poll();
if (result != null) {
limit.release(weigher.apply(result));
}
return result;
}
/**
* Retrieves and removes the head of this queue, waiting up to the specified wait time if
* necessary for an element to become available.
*
* @param timeout how long to wait before giving up, in units of {@code unit}
* @param unit a {@code TimeUnit} determining how to interpret the {@code timeout} parameter
* @return the head of this queue, or {@code null} if the specified waiting time elapses before
* an element is available
* @throws InterruptedException if interrupted while waiting
*/
public @Nullable V poll(long timeout, TimeUnit unit) throws InterruptedException {
V result = queue.poll(timeout, unit);
if (result != null) {
limit.release(weigher.apply(result));
}
return result;
}
/** Returns and removes the next value, or blocks until one is available. */
public @Nullable V take() throws InterruptedException {
V result = queue.take();
limit.release(weigher.apply(result));
return result;
}
/** Returns the current weight of the queue. */
public int weight() {
return maxWeight - limit.availablePermits();
}
public int size() {
return queue.size();
}
}
// Value class for a queued commit.
static class Commit {
private Windmill.WorkItemCommitRequest request;
private ComputationState computationState;
private Work work;
public Commit(
Windmill.WorkItemCommitRequest request, ComputationState computationState, Work work) {
this.request = request;
assert request.getSerializedSize() > 0;
this.computationState = computationState;
this.work = work;
}
public Windmill.WorkItemCommitRequest getRequest() {
return request;
}
public ComputationState getComputationState() {
return computationState;
}
public Work getWork() {
return work;
}
public int getSize() {
return request.getSerializedSize();
}
}
// Maps from computation ids to per-computation state.
private final ConcurrentMap<String, ComputationState> computationMap = new ConcurrentHashMap<>();
private final WeightedBoundedQueue<Commit> commitQueue =
new WeightedBoundedQueue<>(
MAX_COMMIT_QUEUE_BYTES, commit -> Math.min(MAX_COMMIT_QUEUE_BYTES, commit.getSize()));
// Cache of tokens to commit callbacks.
// Using Cache with time eviction policy helps us to prevent memory leak when callback ids are
// discarded by Dataflow service and calling commitCallback is best-effort.
private final Cache<Long, Runnable> commitCallbacks =
CacheBuilder.newBuilder().expireAfterWrite(5L, TimeUnit.MINUTES).build();
// Map of user state names to system state names.
// TODO(drieber): obsolete stateNameMap. Use transformUserNameToStateFamily in
// ComputationState instead.
private final ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
private final ConcurrentMap<String, String> systemNameToComputationIdMap =
new ConcurrentHashMap<>();
final WindmillStateCache stateCache;
private final ThreadFactory threadFactory;
private DataflowMapTaskExecutorFactory mapTaskExecutorFactory;
private final BoundedQueueExecutor workUnitExecutor;
private final WindmillServerStub windmillServer;
private final Thread dispatchThread;
private final Thread commitThread;
private final AtomicLong activeCommitBytes = new AtomicLong();
private final AtomicBoolean running = new AtomicBoolean();
private final StateFetcher stateFetcher;
private final StreamingDataflowWorkerOptions options;
private final boolean windmillServiceEnabled;
private final long clientId;
private final MetricTrackingWindmillServerStub metricTrackingWindmillServer;
private final CounterSet pendingDeltaCounters = new CounterSet();
private final CounterSet pendingCumulativeCounters = new CounterSet();
private final java.util.concurrent.ConcurrentLinkedQueue<CounterUpdate> pendingMonitoringInfos =
new ConcurrentLinkedQueue<>();
// Map from stage name to StageInfo containing metrics container registry and per stage counters.
private final ConcurrentMap<String, StageInfo> stageInfoMap = new ConcurrentHashMap();
// Built-in delta counters.
private final Counter<Long, Long> windmillShuffleBytesRead;
private final Counter<Long, Long> windmillStateBytesRead;
private final Counter<Long, Long> windmillStateBytesWritten;
private final Counter<Long, Long> windmillQuotaThrottling;
// Built-in cumulative counters.
private final Counter<Long, Long> javaHarnessUsedMemory;
private final Counter<Long, Long> javaHarnessMaxMemory;
private final Counter<Integer, Integer> windmillMaxObservedWorkItemCommitBytes;
private final Counter<Integer, Integer> memoryThrashing;
private ScheduledExecutorService refreshWorkTimer;
private ScheduledExecutorService statusPageTimer;
private final boolean publishCounters;
private ScheduledExecutorService globalWorkerUpdatesTimer;
private int retryLocallyDelayMs = 10000;
// Periodically fires a global config request to dataflow service. Only used when windmill service
// is enabled.
private ScheduledExecutorService globalConfigRefreshTimer;
private final MemoryMonitor memoryMonitor;
private final Thread memoryMonitorThread;
private final WorkerStatusPages statusPages;
// Periodic sender of debug information to the debug capture service.
private DebugCapture.Manager debugCaptureManager = null;
// Limit on bytes sinked (committed) in a work item.
private final long maxSinkBytes; // = MAX_SINK_BYTES unless disabled in options.
// Possibly overridden by streaming engine config.
private int maxWorkItemCommitBytes = Integer.MAX_VALUE;
private final EvictingQueue<String> pendingFailuresToReport =
EvictingQueue.<String>create(MAX_FAILURES_TO_REPORT_IN_UPDATE);
private final ReaderCache readerCache;
private final WorkUnitClient workUnitClient;
private final CompletableFuture<Void> isDoneFuture;
private final Function<MapTask, MutableNetwork<Node, Edge>> mapTaskToNetwork;
/**
* Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked
* (across all the sinks, if there are more than one) reaches this limit. This serves as hint for
* readers to stop producing more. This can be disabled with 'disable_limiting_bundle_sink_bytes'
* experiment.
*/
static final int MAX_SINK_BYTES = 10_000_000;
private final ReaderRegistry readerRegistry = ReaderRegistry.defaultRegistry();
private final SinkRegistry sinkRegistry = SinkRegistry.defaultRegistry();
private HotKeyLogger hotKeyLogger;
private final Supplier<Instant> clock;
private final Function<String, ScheduledExecutorService> executorSupplier;
/** Contains a few of the stage specific fields. E.g. metrics container registry, counters etc. */
private static class StageInfo {
final String stageName;
final String systemName;
final MetricsContainerRegistry<StreamingStepMetricsContainer> metricsContainerRegistry;
final StreamingModeExecutionStateRegistry executionStateRegistry;
final CounterSet deltaCounters;
final Counter<Long, Long> throttledMsecs;
final Counter<Long, Long> totalProcessingMsecs;
final Counter<Long, Long> timerProcessingMsecs;
StageInfo(String stageName, String systemName, StreamingDataflowWorker worker) {
this.stageName = stageName;
this.systemName = systemName;
metricsContainerRegistry = StreamingStepMetricsContainer.createRegistry();
executionStateRegistry = new StreamingModeExecutionStateRegistry(worker);
NameContext nameContext = NameContext.create(stageName, null, systemName, null);
deltaCounters = new CounterSet();
throttledMsecs =
deltaCounters.longSum(
StreamingPerStageSystemCounterNames.THROTTLED_MSECS.counterName(nameContext));
totalProcessingMsecs =
deltaCounters.longSum(
StreamingPerStageSystemCounterNames.TOTAL_PROCESSING_MSECS.counterName(nameContext));
timerProcessingMsecs =
deltaCounters.longSum(
StreamingPerStageSystemCounterNames.TIMER_PROCESSING_MSECS.counterName(nameContext));
}
List<CounterUpdate> extractCounterUpdates() {
List<CounterUpdate> counterUpdates = new ArrayList<>();
Iterables.addAll(
counterUpdates,
StreamingStepMetricsContainer.extractMetricUpdates(metricsContainerRegistry));
Iterables.addAll(counterUpdates, executionStateRegistry.extractUpdates(false));
for (CounterUpdate counterUpdate : counterUpdates) {
translateKnownStepCounters(counterUpdate);
}
counterUpdates.addAll(
deltaCounters.extractModifiedDeltaUpdates(DataflowCounterUpdateExtractor.INSTANCE));
return counterUpdates;
}
// Checks if the step counter affects any per-stage counters. Currently 'throttled_millis'
// is the only counter updated.
private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) {
CounterStructuredName structuredName =
stepCounterUpdate.getStructuredNameAndMetadata().getName();
if ((THROTTLING_MSECS_METRIC_NAME.getNamespace().equals(structuredName.getOriginNamespace())
&& THROTTLING_MSECS_METRIC_NAME.getName().equals(structuredName.getName()))
|| (BIGQUERY_STREAMING_INSERT_THROTTLE_TIME
.getNamespace()
.equals(structuredName.getOriginNamespace())
&& BIGQUERY_STREAMING_INSERT_THROTTLE_TIME
.getName()
.equals(structuredName.getName()))) {
long msecs = DataflowCounterUpdateExtractor.splitIntToLong(stepCounterUpdate.getInteger());
if (msecs > 0) {
throttledMsecs.addValue(msecs);
}
}
}
}
public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions(
DataflowWorkerHarnessOptions options) throws IOException {
return new StreamingDataflowWorker(
Collections.emptyList(),
IntrinsicMapTaskExecutorFactory.defaultFactory(),
new DataflowWorkUnitClient(options, LOG),
options.as(StreamingDataflowWorkerOptions.class),
true,
new HotKeyLogger(),
Instant::now,
(threadName) ->
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat(threadName).build()));
}
@VisibleForTesting
StreamingDataflowWorker(
List<MapTask> mapTasks,
DataflowMapTaskExecutorFactory mapTaskExecutorFactory,
WorkUnitClient workUnitClient,
StreamingDataflowWorkerOptions options,
boolean publishCounters,
HotKeyLogger hotKeyLogger,
Supplier<Instant> clock,
Function<String, ScheduledExecutorService> executorSupplier)
throws IOException {
this.stateCache = new WindmillStateCache(options.getWorkerCacheMb());
this.readerCache =
new ReaderCache(
Duration.standardSeconds(options.getReaderCacheTimeoutSec()),
Executors.newCachedThreadPool());
this.mapTaskExecutorFactory = mapTaskExecutorFactory;
this.workUnitClient = workUnitClient;
this.options = options;
this.hotKeyLogger = hotKeyLogger;
this.clock = clock;
this.executorSupplier = executorSupplier;
this.windmillServiceEnabled = options.isEnableStreamingEngine();
this.memoryMonitor = MemoryMonitor.fromOptions(options);
this.statusPages = WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor, () -> true);
if (windmillServiceEnabled) {
this.debugCaptureManager =
new DebugCapture.Manager(options, statusPages.getDebugCapturePages());
}
this.windmillShuffleBytesRead =
pendingDeltaCounters.longSum(
StreamingSystemCounterNames.WINDMILL_SHUFFLE_BYTES_READ.counterName());
this.windmillStateBytesRead =
pendingDeltaCounters.longSum(
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_READ.counterName());
this.windmillStateBytesWritten =
pendingDeltaCounters.longSum(
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_WRITTEN.counterName());
this.windmillQuotaThrottling =
pendingDeltaCounters.longSum(
StreamingSystemCounterNames.WINDMILL_QUOTA_THROTTLING.counterName());
this.javaHarnessUsedMemory =
pendingCumulativeCounters.longSum(
StreamingSystemCounterNames.JAVA_HARNESS_USED_MEMORY.counterName());
this.javaHarnessMaxMemory =
pendingCumulativeCounters.longSum(
StreamingSystemCounterNames.JAVA_HARNESS_MAX_MEMORY.counterName());
this.windmillMaxObservedWorkItemCommitBytes =
pendingCumulativeCounters.intMax(
StreamingSystemCounterNames.WINDMILL_MAX_WORK_ITEM_COMMIT_BYTES.counterName());
this.memoryThrashing =
pendingCumulativeCounters.intSum(
StreamingSystemCounterNames.MEMORY_THRASHING.counterName());
this.isDoneFuture = new CompletableFuture<>();
this.threadFactory =
new ThreadFactoryBuilder().setNameFormat("DataflowWorkUnits-%d").setDaemon(true).build();
this.workUnitExecutor =
new BoundedQueueExecutor(
chooseMaximumNumberOfThreads(),
THREAD_EXPIRATION_TIME_SEC,
TimeUnit.SECONDS,
chooseMaximumBundlesOutstanding(),
chooseMaximumBytesOutstanding(),
threadFactory);
maxSinkBytes =
hasExperiment(options, "disable_limiting_bundle_sink_bytes")
? Long.MAX_VALUE
: MAX_SINK_BYTES;
memoryMonitorThread = new Thread(memoryMonitor);
memoryMonitorThread.setPriority(Thread.MIN_PRIORITY);
memoryMonitorThread.setName("MemoryMonitor");
dispatchThread =
new Thread(
new Runnable() {
@Override
public void run() {
LOG.info("Dispatch starting");
if (windmillServiceEnabled) {
streamingDispatchLoop();
} else {
dispatchLoop();
}
LOG.info("Dispatch done");
}
});
dispatchThread.setDaemon(true);
dispatchThread.setPriority(Thread.MIN_PRIORITY);
dispatchThread.setName("DispatchThread");
commitThread =
new Thread(
new Runnable() {
@Override
public void run() {
if (windmillServiceEnabled) {
streamingCommitLoop();
} else {
commitLoop();
}
}
});
commitThread.setDaemon(true);
commitThread.setPriority(Thread.MAX_PRIORITY);
commitThread.setName("CommitThread");
this.publishCounters = publishCounters;
this.windmillServer = options.getWindmillServerStub();
this.metricTrackingWindmillServer =
new MetricTrackingWindmillServerStub(windmillServer, memoryMonitor, windmillServiceEnabled);
this.metricTrackingWindmillServer.start();
this.stateFetcher = new StateFetcher(metricTrackingWindmillServer);
this.clientId = clientIdGenerator.nextLong();
for (MapTask mapTask : mapTasks) {
addComputation(mapTask.getSystemName(), mapTask, ImmutableMap.of());
}
// Register standard file systems.
FileSystems.setDefaultPipelineOptions(options);
this.mapTaskToNetwork = mapTaskToBaseNetwork;
LOG.debug("windmillServiceEnabled: {}", windmillServiceEnabled);
LOG.debug("WindmillServiceEndpoint: {}", options.getWindmillServiceEndpoint());
LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort());
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes);
}
private int chooseMaximumNumberOfThreads() {
if (options.getNumberOfWorkerHarnessThreads() != 0) {
return options.getNumberOfWorkerHarnessThreads();
}
return MAX_PROCESSING_THREADS;
}
private int chooseMaximumBundlesOutstanding() {
int maxBundles = options.getMaxBundlesFromWindmillOutstanding();
if (maxBundles > 0) {
return maxBundles;
}
return chooseMaximumNumberOfThreads() + 100;
}
private long chooseMaximumBytesOutstanding() {
long maxMem = options.getMaxBytesFromWindmillOutstanding();
if (maxMem > 0) {
return maxMem;
}
return Runtime.getRuntime().maxMemory() / 2;
}
void addStateNameMappings(Map<String, String> nameMap) {
stateNameMap.putAll(nameMap);
}
@VisibleForTesting
public void setRetryLocallyDelayMs(int retryLocallyDelayMs) {
this.retryLocallyDelayMs = retryLocallyDelayMs;
}
@VisibleForTesting
public void setMaxWorkItemCommitBytes(int maxWorkItemCommitBytes) {
if (maxWorkItemCommitBytes != this.maxWorkItemCommitBytes) {
LOG.info("Setting maxWorkItemCommitBytes to {}", maxWorkItemCommitBytes);
}
this.maxWorkItemCommitBytes = maxWorkItemCommitBytes;
}
@VisibleForTesting
public boolean workExecutorIsEmpty() {
return workUnitExecutor.executorQueueIsEmpty();
}
@SuppressWarnings("FutureReturnValueIgnored")
public void start() {
running.set(true);
if (windmillServiceEnabled) {
// Schedule the background getConfig thread. Blocks until windmillServer stub is ready.
schedulePeriodicGlobalConfigRequests();
}
memoryMonitorThread.start();
dispatchThread.start();
commitThread.start();
ExecutionStateSampler.instance().start();
// Periodically report workers counters and other updates.
globalWorkerUpdatesTimer = executorSupplier.apply("GlobalWorkerUpdatesTimer");
globalWorkerUpdatesTimer.scheduleWithFixedDelay(
this::reportPeriodicWorkerUpdates,
0,
options.getWindmillHarnessUpdateReportingPeriod().getMillis(),
TimeUnit.MILLISECONDS);
refreshWorkTimer = executorSupplier.apply("RefreshWork");
if (options.getActiveWorkRefreshPeriodMillis() > 0) {
refreshWorkTimer.scheduleWithFixedDelay(
new Runnable() {
@Override
public void run() {
try {
refreshActiveWork();
} catch (RuntimeException e) {
LOG.warn("Failed to refresh active work: ", e);
}
}
},
options.getActiveWorkRefreshPeriodMillis(),
options.getActiveWorkRefreshPeriodMillis(),
TimeUnit.MILLISECONDS);
}
if (windmillServiceEnabled && options.getStuckCommitDurationMillis() > 0) {
int periodMillis = Math.max(options.getStuckCommitDurationMillis() / 10, 100);
refreshWorkTimer.scheduleWithFixedDelay(
this::invalidateStuckCommits, periodMillis, periodMillis, TimeUnit.MILLISECONDS);
}
if (options.getPeriodicStatusPageOutputDirectory() != null) {
statusPageTimer = executorSupplier.apply("DumpStatusPages");
statusPageTimer.scheduleWithFixedDelay(
() -> {
Collection<Capturable> pages = statusPages.getDebugCapturePages();
if (pages.isEmpty()) {
LOG.warn("No captured status pages.");
}
Long timestamp = clock.get().getMillis();
for (Capturable page : pages) {
PrintWriter writer = null;
try {
File outputFile =
new File(
options.getPeriodicStatusPageOutputDirectory(),
("StreamingDataflowWorker"
+ options.getWorkerId()
+ "_"
+ page.pageName()
+ timestamp.toString())
.replaceAll("/", "_"));
writer = new PrintWriter(outputFile, UTF_8.name());
page.captureData(writer);
} catch (IOException e) {
LOG.warn("Error dumping status page.", e);
} finally {
if (writer != null) {
writer.close();
}
}
}
},
60,
60,
TimeUnit.SECONDS);
}
reportHarnessStartup();
}
public void startStatusPages() {
if (debugCaptureManager != null) {
debugCaptureManager.start();
}
statusPages.addServlet(stateCache.statusServlet());
statusPages.addServlet(new SpecsServlet());
statusPages.addStatusDataProvider("harness", "Harness", new HarnessDataProvider());
statusPages.addStatusDataProvider("metrics", "Metrics", new MetricsDataProvider());
statusPages.addStatusDataProvider(
"exception", "Last Exception", new LastExceptionDataProvider());
statusPages.addStatusDataProvider("cache", "State Cache", stateCache);
statusPages.addStatusDataProvider("streaming", "Streaming Rpcs", windmillServer);
statusPages.start();
}
public void addWorkerStatusPage(BaseStatusServlet page) {
statusPages.addServlet(page);
if (page instanceof Capturable) {
statusPages.addCapturePage((Capturable) page);
}
}
public void stop() {
try {
if (globalConfigRefreshTimer != null) {
globalConfigRefreshTimer.shutdown();
}
globalWorkerUpdatesTimer.shutdown();
if (refreshWorkTimer != null) {
refreshWorkTimer.shutdown();
}
if (statusPageTimer != null) {
statusPageTimer.shutdown();
}
if (globalConfigRefreshTimer != null) {
globalConfigRefreshTimer.awaitTermination(300, TimeUnit.SECONDS);
}
globalWorkerUpdatesTimer.awaitTermination(300, TimeUnit.SECONDS);
if (refreshWorkTimer != null) {
refreshWorkTimer.awaitTermination(300, TimeUnit.SECONDS);
}
if (statusPageTimer != null) {
statusPageTimer.awaitTermination(300, TimeUnit.SECONDS);
}
statusPages.stop();
if (debugCaptureManager != null) {
debugCaptureManager.stop();
}
running.set(false);
dispatchThread.interrupt();
dispatchThread.join();
// We need to interrupt the commitThread in case it is blocking on pulling
// from the commitQueue.
commitThread.interrupt();
commitThread.join();
memoryMonitor.stop();
memoryMonitorThread.join();
workUnitExecutor.shutdown();
for (ComputationState state : computationMap.values()) {
state.close();
}
// one last send
reportPeriodicWorkerUpdates();
} catch (Exception e) {
LOG.warn("Exception while shutting down: ", e);
}
setIsDone();
}
// null is the only value of type Void, but findbugs thinks
// it violates the contract of CompletableFuture.complete
@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
private void setIsDone() {
isDoneFuture.complete(null);
}
public void waitTillExecutionFinishes() throws InterruptedException {
try {
isDoneFuture.get();
} catch (ExecutionException e) {
throw new IllegalStateException(e);
}
}
private synchronized void addComputation(
String computationId,
MapTask originalMapTask,
Map<String, String> transformUserNameToStateFamily) {
// Map task instances are shared amongst multiple threads during computation hence
// we fix the map task before we add a new computation state that would reference it.
MapTask mapTask = fixMultiOutputInfos.apply(originalMapTask);
if (!computationMap.containsKey(computationId)) {
LOG.info("Adding config for {}: {}", computationId, mapTask);
computationMap.put(
computationId,
new ComputationState(
computationId,
mapTask,
workUnitExecutor,
transformUserNameToStateFamily,
stateCache.forComputation(computationId)));
}
}
private static void sleep(int millis) {
Uninterruptibles.sleepUninterruptibly(millis, TimeUnit.MILLISECONDS);
}
/**
* If the computation is not yet known about, configuration for it will be fetched. This can still
* return null if there is no configuration fetched for the computation.
*/
private ComputationState getComputationState(String computationId) {
ComputationState state = computationMap.get(computationId);
if (state == null) {
getConfig(computationId);
state = computationMap.get(computationId);
}
return state;
}
private void dispatchLoop() {
while (running.get()) {
memoryMonitor.waitForResources("GetWork");
int backoff = 1;
Windmill.GetWorkResponse workResponse = null;
do {
try {
workResponse = getWork();
if (workResponse.getWorkCount() > 0) {
break;
}
} catch (WindmillServerStub.RpcException e) {
LOG.warn("GetWork failed, retrying:", e);
}
sleep(backoff);
backoff = Math.min(1000, backoff * 2);
} while (running.get());
for (final Windmill.ComputationWorkItems computationWork : workResponse.getWorkList()) {
final String computationId = computationWork.getComputationId();
final ComputationState computationState = getComputationState(computationId);
if (computationState == null) {
LOG.warn(
"Received work for unknown computation: {}. Known computations are {}",
computationId,
computationMap.keySet());
continue;
}
final Instant inputDataWatermark =
WindmillTimeUtils.windmillToHarnessWatermark(computationWork.getInputDataWatermark());
Preconditions.checkNotNull(inputDataWatermark);
final @Nullable Instant synchronizedProcessingTime =
WindmillTimeUtils.windmillToHarnessWatermark(
computationWork.getDependentRealtimeInputWatermark());
for (final Windmill.WorkItem workItem : computationWork.getWorkList()) {
scheduleWorkItem(
computationState, inputDataWatermark, synchronizedProcessingTime, workItem);
}
}
}
}
void streamingDispatchLoop() {
while (running.get()) {
GetWorkStream stream =
windmillServer.getWorkStream(
Windmill.GetWorkRequest.newBuilder()