dawidwys commented on a change in pull request #18665:
URL: https://github.com/apache/flink/pull/18665#discussion_r801810425
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
##########
@@ -101,16 +115,71 @@ Licensed to the Apache Software Foundation (ASF) under one
public SourceCoordinator(
Review comment:
nit: Could we drop the ctor, already? I believe there are not many
places we would need to change.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
##########
@@ -101,16 +115,71 @@ Licensed to the Apache Software Foundation (ASF) under one
public SourceCoordinator(
String operatorName,
- ExecutorService coordinatorExecutor,
+ ScheduledExecutorService coordinatorExecutor,
Source<?, SplitT, EnumChkT> source,
SourceCoordinatorContext<SplitT> context,
CoordinatorStore coordinatorStore) {
+ this(
+ operatorName,
+ coordinatorExecutor,
+ source,
+ context,
+ coordinatorStore,
+ WATERMARK_ALIGNMENT_DISABLED);
+ }
+
+ public SourceCoordinator(
+ String operatorName,
+ ScheduledExecutorService coordinatorExecutor,
+ Source<?, SplitT, EnumChkT> source,
+ SourceCoordinatorContext<SplitT> context,
+ CoordinatorStore coordinatorStore,
+ WatermarkAlignmentParams watermarkAlignmentParams) {
this.operatorName = operatorName;
this.coordinatorExecutor = coordinatorExecutor;
this.source = source;
this.enumCheckpointSerializer =
source.getEnumeratorCheckpointSerializer();
this.context = context;
this.coordinatorStore = coordinatorStore;
+ this.watermarkAlignmentParams = watermarkAlignmentParams;
+
+ if (watermarkAlignmentParams.isEnabled()) {
+ coordinatorStore.putIfAbsent(
+ watermarkAlignmentParams.watermarkGroup, new
WatermarkAggregator<>());
+ coordinatorExecutor.scheduleAtFixedRate(
+ this::announceCombinedWatermark,
+ watermarkAlignmentParams.updateInterval,
+ watermarkAlignmentParams.updateInterval,
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ @VisibleForTesting
+ void announceCombinedWatermark() {
+ checkState(watermarkAlignmentParams != WATERMARK_ALIGNMENT_DISABLED);
+
+ Watermark globalCombinedWatermark =
+ coordinatorStore.apply(
+ watermarkAlignmentParams.watermarkGroup,
+ (value) -> {
+ WatermarkAggregator aggregator =
(WatermarkAggregator) value;
+ return new Watermark(
+
aggregator.getAggregatedWatermark().getTimestamp());
+ });
+
+ // TODO: check if min actually changed
Review comment:
nit: Can we remove the `TODO`?
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorAlignmentTest.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.runtime.source.coordinator;
+
+import org.apache.flink.core.fs.AutoCloseableRegistry;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import
org.apache.flink.runtime.source.coordinator.SourceCoordinator.WatermarkAlignmentParams;
+import org.apache.flink.runtime.source.event.ReportedWatermarkEvent;
+import org.apache.flink.runtime.source.event.WatermarkAlignmentEvent;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/** Unit tests for watermark alignment of the {@link SourceCoordinator}. */
+@SuppressWarnings("serial")
+public class SourceCoordinatorAlignmentTest extends SourceCoordinatorTestBase {
+
+ @Test
+ public void testWatermarkAlignment() throws Exception {
+ try (AutoCloseableRegistry closeableRegistry = new
AutoCloseableRegistry()) {
+ SourceCoordinator<?, ?> sourceCoordinator1 =
+ getAndStartNewSourceCoordinator(
+ new WatermarkAlignmentParams(1000L, "group1",
Long.MAX_VALUE),
+ closeableRegistry);
+
+ reportWatermarkEvent(sourceCoordinator1, 0, 42);
Review comment:
nit: could we extract subtask ids to variables? Just for the sake of
readability:
```
reportWatermarkEvent(sourceCoordinator1, subtask0, 42);
```
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
##########
@@ -440,9 +513,80 @@ private void
handleReaderRegistrationEvent(ReaderRegistrationEvent event) {
enumerator.addReader(event.subtaskId());
}
+ private void handleReportedWatermark(int subtask, Watermark watermark) {
+ LOG.info("New reported watermark={} from subTaskId={}", watermark,
subtask);
+
+ checkState(watermarkAlignmentParams != WATERMARK_ALIGNMENT_DISABLED);
+
+ combinedWatermark
+ .aggregate(subtask, watermark)
+ .ifPresent(
+ newCombinedWatermark ->
+ coordinatorStore.computeIfPresent(
+
watermarkAlignmentParams.watermarkGroup,
+ (key, oldValue) -> {
+ WatermarkAggregator
watermarkAggregator =
+ (WatermarkAggregator)
oldValue;
Review comment:
```suggestion
WatermarkAggregator<?>
watermarkAggregator =
(WatermarkAggregator<?>)
oldValue;
```
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
##########
@@ -440,9 +513,80 @@ private void
handleReaderRegistrationEvent(ReaderRegistrationEvent event) {
enumerator.addReader(event.subtaskId());
}
+ private void handleReportedWatermark(int subtask, Watermark watermark) {
+ LOG.info("New reported watermark={} from subTaskId={}", watermark,
subtask);
+
+ checkState(watermarkAlignmentParams != WATERMARK_ALIGNMENT_DISABLED);
Review comment:
`watermarkAlignmentParams.isEnabled()`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
##########
@@ -440,9 +513,80 @@ private void
handleReaderRegistrationEvent(ReaderRegistrationEvent event) {
enumerator.addReader(event.subtaskId());
}
+ private void handleReportedWatermark(int subtask, Watermark watermark) {
+ LOG.info("New reported watermark={} from subTaskId={}", watermark,
subtask);
Review comment:
`debug`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/event/WatermarkAlignmentEvent.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.source.event;
+
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+
+import java.util.Objects;
+
+/** Signals the reader the maximum watermark that emitted records should have.
*/
Review comment:
```suggestion
/** Signals source operators the maximum watermark that emitted records can
have. */
```
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/source/event/ReportedWatermarkEvent.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.runtime.source.event;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+
+import java.util.Objects;
+
+/**
+ * Reports {@link Watermark} from a subtask to the {@link
Review comment:
```suggestion
* Reports last emitted {@link Watermark} from a subtask to the {@link
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]