yunfengzhou-hub commented on code in PR #20275: URL: https://github.com/apache/flink/pull/20275#discussion_r931731203
########## flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java: ########## @@ -0,0 +1,350 @@ +/* + * 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.runtime.tasks; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.testutils.MiniClusterResource; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Random; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent; +import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator; +import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent; +import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration test case that validates the exactly-once mechanism for operator events sent around + * checkpoint. This class is an extension to {@link + * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further + * verifying the exactly-once semantics of events when the flink job is constructed using actual + * stream operators. + */ +public class CoordinatorEventsExactlyOnceTest { Review Comment: Because the implementations of this PR are in `flink-runtime` and `flink-streaming-java`. Given that `flink-streaming-java` depends on `flink-runtime`, an e2e test for the problem solved in this PR should be in `flink-streaming-java`. As already explained in the JavaDoc, The newly added test class verifies the correctness of coordinator events with actual stream operators, while the existing `CoordinatorEventsExactlyOnceITCase` makes the recipient of coordinator events a general `AbstractInvokable` subclass. The scope of these two test classes is different. How do you like it to remove `CoordinatorEventsExactlyOnceITCase`? I now think that the `AbstractInvokable` may not provide more verifications than a `StreamOperator`, so the test case can be removed. If we agree on this idea, there will not be two integration tests in two packages about a similar situation, and the naming problem would not exist. -- 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]
