dawidwys commented on a change in pull request #18702:
URL: https://github.com/apache/flink/pull/18702#discussion_r803680107



##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorAlignmentTest.java
##########
@@ -0,0 +1,155 @@
+/*
+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.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.eventtime.WatermarkGenerator;
+import org.apache.flink.api.common.eventtime.WatermarkOutput;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import 
org.apache.flink.runtime.source.coordinator.SourceCoordinator.WatermarkAlignmentParams;
+import org.apache.flink.runtime.source.event.AddSplitEvent;
+import org.apache.flink.runtime.source.event.ReportedWatermarkEvent;
+import org.apache.flink.runtime.source.event.WatermarkAlignmentEvent;
+import org.apache.flink.streaming.api.operators.source.CollectingDataOutput;
+import org.apache.flink.streaming.runtime.io.DataInputStatus;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Unit test for {@link SourceOperator} watermark alignment. */
+@SuppressWarnings("serial")
+public class SourceOperatorAlignmentTest {

Review comment:
       Could we update it to JUnit 5?

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorAlignmentTest.java
##########
@@ -0,0 +1,155 @@
+/*
+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.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.eventtime.WatermarkGenerator;
+import org.apache.flink.api.common.eventtime.WatermarkOutput;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import 
org.apache.flink.runtime.source.coordinator.SourceCoordinator.WatermarkAlignmentParams;
+import org.apache.flink.runtime.source.event.AddSplitEvent;
+import org.apache.flink.runtime.source.event.ReportedWatermarkEvent;
+import org.apache.flink.runtime.source.event.WatermarkAlignmentEvent;
+import org.apache.flink.streaming.api.operators.source.CollectingDataOutput;
+import org.apache.flink.streaming.runtime.io.DataInputStatus;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Unit test for {@link SourceOperator} watermark alignment. */
+@SuppressWarnings("serial")
+public class SourceOperatorAlignmentTest {
+
+    @Nullable private SourceOperatorTestContext context;
+    @Nullable private SourceOperator<Integer, MockSourceSplit> operator;
+
+    @Before
+    public void setup() throws Exception {
+        context =
+                new SourceOperatorTestContext(
+                        false,
+                        WatermarkStrategy.forGenerator(ctx -> new 
PunctuatedGenerator())
+                                .withTimestampAssigner((r, t) -> r),
+                        new WatermarkAlignmentParams(100, "group1", 1));
+        operator = context.getOperator();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        context.close();
+        context = null;
+        operator = null;
+    }
+
+    @Test
+    public void testHandleAddSplitsEvent() throws Exception {

Review comment:
       I guess we should update the test name ;)

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/CollectingDataOutput.java
##########
@@ -47,11 +47,16 @@ public void emitWatermarkStatus(WatermarkStatus 
watermarkStatus) throws Exceptio
 
     @Override
     public void emitRecord(StreamRecord<E> streamRecord) throws Exception {
-        events.add(streamRecord);
+        // Bypass issues with object re-use disabled by copying the record
+        events.add(streamRecord.copy(streamRecord.getValue()));

Review comment:
       out of curiosity, what was the issue here?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
##########
@@ -370,7 +413,14 @@ public DataInputStatus emitNext(DataOutput<OUT> output) 
throws Exception {
     private DataInputStatus emitNextNotReading(DataOutput<OUT> output) throws 
Exception {
         switch (operatingMode) {
             case OUTPUT_NOT_INITIALIZED:
-                currentMainOutput = eventTimeLogic.createMainOutput(output);
+                watermarkTrackingOutput = new 
WatermarkTrackingOutput<>(output);
+                if (watermarkAlignmentParams.isEnabled()) {
+                    processingTimeService.scheduleWithFixedDelay(
+                            this::emitLatestWatermark,
+                            watermarkAlignmentParams.getUpdateInterval(),
+                            watermarkAlignmentParams.getUpdateInterval());
+                }
+                currentMainOutput = 
eventTimeLogic.createMainOutput(watermarkTrackingOutput);

Review comment:
       thiis will most probably cause some tests to fail. 
   
   We should use the original `output` if watermark alignment is disabled.




-- 
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]


Reply via email to