forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 7
/
AbstractStreamOperator.java
676 lines (587 loc) · 27.3 KB
/
AbstractStreamOperator.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
/*
* 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.flink.streaming.api.operators;
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.eventtime.IndexedCombinedWatermarkStatus;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.api.common.state.State;
import org.apache.flink.api.common.state.StateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.MetricOptions;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.core.memory.ManagedMemoryUseCase;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.checkpoint.CheckpointOptions;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.KeyedStateBackend;
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.runtime.state.VoidNamespace;
import org.apache.flink.runtime.state.VoidNamespaceSerializer;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.CheckpointedStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.util.LatencyStats;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Serializable;
import java.util.Locale;
import java.util.Optional;
import static org.apache.flink.util.Preconditions.checkState;
/**
* Base class for all stream operators. Operators that contain a user function should extend the
* class {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class).
*
* <p>For concrete implementations, one of the following two interfaces must also be implemented, to
* mark the operator as unary or binary: {@link OneInputStreamOperator} or {@link
* TwoInputStreamOperator}.
*
* <p>Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
* the timer service, timer callbacks are also guaranteed not to be called concurrently with methods
* on {@code StreamOperator}.
*
* <p>Note, this class is going to be removed and replaced in the future by {@link
* AbstractStreamOperatorV2}. However as {@link AbstractStreamOperatorV2} is currently experimental,
* {@link AbstractStreamOperator} has not been deprecated just yet.
*
* @param <OUT> The output type of the operator.
*/
@PublicEvolving
public abstract class AbstractStreamOperator<OUT>
implements StreamOperator<OUT>,
SetupableStreamOperator<OUT>,
CheckpointedStreamOperator,
Serializable {
private static final long serialVersionUID = 1L;
/** The logger used by the operator class and its subclasses. */
protected static final Logger LOG = LoggerFactory.getLogger(AbstractStreamOperator.class);
// ----------- configuration properties -------------
// A sane default for most operators
protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
// ---------------- runtime fields ------------------
/** The task that contains this operator (and other operators in the same chain). */
private transient StreamTask<?, ?> container;
protected transient StreamConfig config;
protected transient Output<StreamRecord<OUT>> output;
private transient IndexedCombinedWatermarkStatus combinedWatermark;
/** The runtime context for UDFs. */
private transient StreamingRuntimeContext runtimeContext;
// ---------------- key/value state ------------------
/**
* {@code KeySelector} for extracting a key from an element being processed. This is used to
* scope keyed state to a key. This is null if the operator is not a keyed operator.
*
* <p>This is for elements from the first input.
*/
private transient KeySelector<?, ?> stateKeySelector1;
/**
* {@code KeySelector} for extracting a key from an element being processed. This is used to
* scope keyed state to a key. This is null if the operator is not a keyed operator.
*
* <p>This is for elements from the second input.
*/
private transient KeySelector<?, ?> stateKeySelector2;
private transient StreamOperatorStateHandler stateHandler;
private transient InternalTimeServiceManager<?> timeServiceManager;
// --------------- Metrics ---------------------------
/** Metric group for the operator. */
protected transient OperatorMetricGroup metrics;
protected transient LatencyStats latencyStats;
// ---------------- time handler ------------------
protected transient ProcessingTimeService processingTimeService;
// ------------------------------------------------------------------------
// Life Cycle
// ------------------------------------------------------------------------
@Override
public void setup(
StreamTask<?, ?> containingTask,
StreamConfig config,
Output<StreamRecord<OUT>> output) {
final Environment environment = containingTask.getEnvironment();
this.container = containingTask;
this.config = config;
try {
OperatorMetricGroup operatorMetricGroup =
environment
.getMetricGroup()
.getOrAddOperator(config.getOperatorID(), config.getOperatorName());
this.output =
new CountingOutput<>(
output,
operatorMetricGroup.getIOMetricGroup().getNumRecordsOutCounter());
if (config.isChainEnd()) {
operatorMetricGroup.getIOMetricGroup().reuseOutputMetricsForTask();
}
this.metrics = operatorMetricGroup;
} catch (Exception e) {
LOG.warn("An error occurred while instantiating task metrics.", e);
this.metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup();
this.output = output;
}
this.combinedWatermark = IndexedCombinedWatermarkStatus.forInputsCount(2);
try {
Configuration taskManagerConfig = environment.getTaskManagerInfo().getConfiguration();
int historySize = taskManagerConfig.getInteger(MetricOptions.LATENCY_HISTORY_SIZE);
if (historySize <= 0) {
LOG.warn(
"{} has been set to a value equal or below 0: {}. Using default.",
MetricOptions.LATENCY_HISTORY_SIZE,
historySize);
historySize = MetricOptions.LATENCY_HISTORY_SIZE.defaultValue();
}
final String configuredGranularity =
taskManagerConfig.getString(MetricOptions.LATENCY_SOURCE_GRANULARITY);
LatencyStats.Granularity granularity;
try {
granularity =
LatencyStats.Granularity.valueOf(
configuredGranularity.toUpperCase(Locale.ROOT));
} catch (IllegalArgumentException iae) {
granularity = LatencyStats.Granularity.OPERATOR;
LOG.warn(
"Configured value {} option for {} is invalid. Defaulting to {}.",
configuredGranularity,
MetricOptions.LATENCY_SOURCE_GRANULARITY.key(),
granularity);
}
TaskManagerJobMetricGroup jobMetricGroup = this.metrics.parent().parent();
this.latencyStats =
new LatencyStats(
jobMetricGroup.addGroup("latency"),
historySize,
container.getIndexInSubtaskGroup(),
getOperatorID(),
granularity);
} catch (Exception e) {
LOG.warn("An error occurred while instantiating latency metrics.", e);
this.latencyStats =
new LatencyStats(
UnregisteredMetricGroups.createUnregisteredTaskManagerJobMetricGroup()
.addGroup("latency"),
1,
0,
new OperatorID(),
LatencyStats.Granularity.SINGLE);
}
this.runtimeContext =
new StreamingRuntimeContext(
environment,
environment.getAccumulatorRegistry().getUserMap(),
getMetricGroup(),
getOperatorID(),
getProcessingTimeService(),
null,
environment.getExternalResourceInfoProvider());
stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader());
stateKeySelector2 = config.getStatePartitioner(1, getUserCodeClassloader());
}
/**
* @deprecated The {@link ProcessingTimeService} instance should be passed by the operator
* constructor and this method will be removed along with {@link SetupableStreamOperator}.
*/
@Deprecated
public void setProcessingTimeService(ProcessingTimeService processingTimeService) {
this.processingTimeService = Preconditions.checkNotNull(processingTimeService);
}
@Override
public MetricGroup getMetricGroup() {
return metrics;
}
@Override
public final void initializeState(StreamTaskStateInitializer streamTaskStateManager)
throws Exception {
final TypeSerializer<?> keySerializer =
config.getStateKeySerializer(getUserCodeClassloader());
final StreamTask<?, ?> containingTask = Preconditions.checkNotNull(getContainingTask());
final CloseableRegistry streamTaskCloseableRegistry =
Preconditions.checkNotNull(containingTask.getCancelables());
final StreamOperatorStateContext context =
streamTaskStateManager.streamOperatorStateContext(
getOperatorID(),
getClass().getSimpleName(),
getProcessingTimeService(),
this,
keySerializer,
streamTaskCloseableRegistry,
metrics,
config.getManagedMemoryFractionOperatorUseCaseOfSlot(
ManagedMemoryUseCase.STATE_BACKEND,
runtimeContext.getTaskManagerRuntimeInfo().getConfiguration(),
runtimeContext.getUserCodeClassLoader()),
isUsingCustomRawKeyedState());
stateHandler =
new StreamOperatorStateHandler(
context, getExecutionConfig(), streamTaskCloseableRegistry);
timeServiceManager = context.internalTimerServiceManager();
stateHandler.initializeOperatorState(this);
runtimeContext.setKeyedStateStore(stateHandler.getKeyedStateStore().orElse(null));
}
/**
* Indicates whether or not implementations of this class is writing to the raw keyed state
* streams on snapshots, using {@link #snapshotState(StateSnapshotContext)}. If yes, subclasses
* should override this method to return {@code true}.
*
* <p>Subclasses need to explicitly indicate the use of raw keyed state because, internally, the
* {@link AbstractStreamOperator} may attempt to read from it as well to restore heap-based
* timers and ultimately fail with read errors. By setting this flag to {@code true}, this
* allows the {@link AbstractStreamOperator} to know that the data written in the raw keyed
* states were not written by the timer services, and skips the timer restore attempt.
*
* <p>Please refer to FLINK-19741 for further details.
*
* <p>TODO: this method can be removed once all timers are moved to be managed by state
* backends.
*
* @return flag indicating whether or not this operator is writing to raw keyed state via {@link
* #snapshotState(StateSnapshotContext)}.
*/
@Internal
protected boolean isUsingCustomRawKeyedState() {
return false;
}
/**
* This method is called immediately before any elements are processed, it should contain the
* operator's initialization logic, e.g. state initialization.
*
* <p>The default implementation does nothing.
*
* @throws Exception An exception in this method causes the operator to fail.
*/
@Override
public void open() throws Exception {}
/**
* This method is called after all records have been added to the operators via the methods
* {@link OneInputStreamOperator#processElement(StreamRecord)}, or {@link
* TwoInputStreamOperator#processElement1(StreamRecord)} and {@link
* TwoInputStreamOperator#processElement2(StreamRecord)}.
*
* <p>The method is expected to flush all remaining buffered data. Exceptions during this
* flushing of buffered should be propagated, in order to cause the operation to be recognized
* asa failed, because the last data items are not processed properly.
*
* @throws Exception An exception in this method causes the operator to fail.
*/
@Override
public void close() throws Exception {}
/**
* This method is called at the very end of the operator's life, both in the case of a
* successful completion of the operation, and in the case of a failure and canceling.
*
* <p>This method is expected to make a thorough effort to release all resources that the
* operator has acquired.
*/
@Override
public void dispose() throws Exception {
if (stateHandler != null) {
stateHandler.dispose();
}
}
@Override
public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
// the default implementation does nothing and accepts the checkpoint
// this is purely for subclasses to override
}
@Override
public final OperatorSnapshotFutures snapshotState(
long checkpointId,
long timestamp,
CheckpointOptions checkpointOptions,
CheckpointStreamFactory factory)
throws Exception {
return stateHandler.snapshotState(
this,
Optional.ofNullable(timeServiceManager),
getOperatorName(),
checkpointId,
timestamp,
checkpointOptions,
factory,
isUsingCustomRawKeyedState());
}
/**
* Stream operators with state, which want to participate in a snapshot need to override this
* hook method.
*
* @param context context that provides information and means required for taking a snapshot
*/
@Override
public void snapshotState(StateSnapshotContext context) throws Exception {}
/**
* Stream operators with state which can be restored need to override this hook method.
*
* @param context context that allows to register different states.
*/
@Override
public void initializeState(StateInitializationContext context) throws Exception {}
@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
stateHandler.notifyCheckpointComplete(checkpointId);
}
@Override
public void notifyCheckpointAborted(long checkpointId) throws Exception {
stateHandler.notifyCheckpointAborted(checkpointId);
}
// ------------------------------------------------------------------------
// Properties and Services
// ------------------------------------------------------------------------
/**
* Gets the execution config defined on the execution environment of the job to which this
* operator belongs.
*
* @return The job's execution config.
*/
public ExecutionConfig getExecutionConfig() {
return container.getExecutionConfig();
}
public StreamConfig getOperatorConfig() {
return config;
}
public StreamTask<?, ?> getContainingTask() {
return container;
}
public ClassLoader getUserCodeClassloader() {
return container.getUserCodeClassLoader();
}
/**
* Return the operator name. If the runtime context has been set, then the task name with
* subtask index is returned. Otherwise, the simple class name is returned.
*
* @return If runtime context is set, then return task name with subtask index. Otherwise return
* simple class name.
*/
protected String getOperatorName() {
if (runtimeContext != null) {
return runtimeContext.getTaskNameWithSubtasks();
} else {
return getClass().getSimpleName();
}
}
/**
* Returns a context that allows the operator to query information about the execution and also
* to interact with systems such as broadcast variables and managed state. This also allows to
* register timers.
*/
@VisibleForTesting
public StreamingRuntimeContext getRuntimeContext() {
return runtimeContext;
}
@VisibleForTesting
public <K> KeyedStateBackend<K> getKeyedStateBackend() {
return stateHandler.getKeyedStateBackend();
}
@VisibleForTesting
public OperatorStateBackend getOperatorStateBackend() {
return stateHandler.getOperatorStateBackend();
}
/**
* Returns the {@link ProcessingTimeService} responsible for getting the current processing time
* and registering timers.
*/
@VisibleForTesting
public ProcessingTimeService getProcessingTimeService() {
return processingTimeService;
}
/**
* Creates a partitioned state handle, using the state backend configured for this task.
*
* @throws IllegalStateException Thrown, if the key/value state was already initialized.
* @throws Exception Thrown, if the state backend cannot create the key/value state.
*/
protected <S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor)
throws Exception {
return getPartitionedState(
VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, stateDescriptor);
}
protected <N, S extends State, T> S getOrCreateKeyedState(
TypeSerializer<N> namespaceSerializer, StateDescriptor<S, T> stateDescriptor)
throws Exception {
return stateHandler.getOrCreateKeyedState(namespaceSerializer, stateDescriptor);
}
/**
* Creates a partitioned state handle, using the state backend configured for this task.
*
* @throws IllegalStateException Thrown, if the key/value state was already initialized.
* @throws Exception Thrown, if the state backend cannot create the key/value state.
*/
protected <S extends State, N> S getPartitionedState(
N namespace,
TypeSerializer<N> namespaceSerializer,
StateDescriptor<S, ?> stateDescriptor)
throws Exception {
return stateHandler.getPartitionedState(namespace, namespaceSerializer, stateDescriptor);
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void setKeyContextElement1(StreamRecord record) throws Exception {
setKeyContextElement(record, stateKeySelector1);
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void setKeyContextElement2(StreamRecord record) throws Exception {
setKeyContextElement(record, stateKeySelector2);
}
private <T> void setKeyContextElement(StreamRecord<T> record, KeySelector<T, ?> selector)
throws Exception {
if (selector != null) {
Object key = selector.getKey(record.getValue());
setCurrentKey(key);
}
}
public void setCurrentKey(Object key) {
stateHandler.setCurrentKey(key);
}
public Object getCurrentKey() {
return stateHandler.getCurrentKey();
}
public KeyedStateStore getKeyedStateStore() {
if (stateHandler == null) {
return null;
}
return stateHandler.getKeyedStateStore().orElse(null);
}
// ------------------------------------------------------------------------
// Context and chaining properties
// ------------------------------------------------------------------------
@Override
public final void setChainingStrategy(ChainingStrategy strategy) {
this.chainingStrategy = strategy;
}
@Override
public final ChainingStrategy getChainingStrategy() {
return chainingStrategy;
}
// ------------------------------------------------------------------------
// Metrics
// ------------------------------------------------------------------------
// ------- One input stream
public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
reportOrForwardLatencyMarker(latencyMarker);
}
// ------- Two input stream
public void processLatencyMarker1(LatencyMarker latencyMarker) throws Exception {
reportOrForwardLatencyMarker(latencyMarker);
}
public void processLatencyMarker2(LatencyMarker latencyMarker) throws Exception {
reportOrForwardLatencyMarker(latencyMarker);
}
protected void reportOrForwardLatencyMarker(LatencyMarker marker) {
// all operators are tracking latencies
this.latencyStats.reportLatency(marker);
// everything except sinks forwards latency markers
this.output.emitLatencyMarker(marker);
}
// ------------------------------------------------------------------------
// Watermark handling
// ------------------------------------------------------------------------
/**
* Returns a {@link InternalTimerService} that can be used to query current processing time and
* event time and to set timers. An operator can have several timer services, where each has its
* own namespace serializer. Timer services are differentiated by the string key that is given
* when requesting them, if you call this method with the same key multiple times you will get
* the same timer service instance in subsequent requests.
*
* <p>Timers are always scoped to a key, the currently active key of a keyed stream operation.
* When a timer fires, this key will also be set as the currently active key.
*
* <p>Each timer has attached metadata, the namespace. Different timer services can have a
* different namespace type. If you don't need namespace differentiation you can use {@link
* VoidNamespaceSerializer} as the namespace serializer.
*
* @param name The name of the requested timer service. If no service exists under the given
* name a new one will be created and returned.
* @param namespaceSerializer {@code TypeSerializer} for the timer namespace.
* @param triggerable The {@link Triggerable} that should be invoked when timers fire
* @param <N> The type of the timer namespace.
*/
public <K, N> InternalTimerService<N> getInternalTimerService(
String name, TypeSerializer<N> namespaceSerializer, Triggerable<K, N> triggerable) {
if (timeServiceManager == null) {
throw new RuntimeException("The timer service has not been initialized.");
}
@SuppressWarnings("unchecked")
InternalTimeServiceManager<K> keyedTimeServiceHandler =
(InternalTimeServiceManager<K>) timeServiceManager;
KeyedStateBackend<K> keyedStateBackend = getKeyedStateBackend();
checkState(keyedStateBackend != null, "Timers can only be used on keyed operators.");
return keyedTimeServiceHandler.getInternalTimerService(
name, keyedStateBackend.getKeySerializer(), namespaceSerializer, triggerable);
}
public void processWatermark(Watermark mark) throws Exception {
if (timeServiceManager != null) {
timeServiceManager.advanceWatermark(mark);
}
output.emitWatermark(mark);
}
private void processWatermark(Watermark mark, int index) throws Exception {
if (combinedWatermark.updateWatermark(index, mark.getTimestamp())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
}
public void processWatermark1(Watermark mark) throws Exception {
processWatermark(mark, 0);
}
public void processWatermark2(Watermark mark) throws Exception {
processWatermark(mark, 1);
}
public final void emitStreamStatus(StreamStatus streamStatus) throws Exception {
processStreamStatus(streamStatus);
}
private void emitStreamStatus(StreamStatus streamStatus, int index) throws Exception {
boolean wasIdle = combinedWatermark.isIdle();
if (combinedWatermark.updateStatus(index, streamStatus.isIdle())) {
processWatermark(new Watermark(combinedWatermark.getCombinedWatermark()));
}
if (wasIdle != combinedWatermark.isIdle()) {
processStreamStatus(streamStatus);
}
}
public final void emitStreamStatus1(StreamStatus streamStatus) throws Exception {
emitStreamStatus(streamStatus, 0);
}
public final void emitStreamStatus2(StreamStatus streamStatus) throws Exception {
emitStreamStatus(streamStatus, 1);
}
@Override
public void processStreamStatus(StreamStatus status) throws Exception {
output.emitStreamStatus(status);
}
@Override
public OperatorID getOperatorID() {
return config.getOperatorID();
}
protected Optional<InternalTimeServiceManager<?>> getTimeServiceManager() {
return Optional.ofNullable(timeServiceManager);
}
}