pnowojski commented on code in PR #22921: URL: https://github.com/apache/flink/pull/22921#discussion_r1250900302
########## flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorAlignmentBenchmark.java: ########## @@ -0,0 +1,97 @@ +/* + * 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.api.common.eventtime.WatermarkAlignmentParams; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.mocks.MockSource; +import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.source.event.ReportedWatermarkEvent; + +import javax.annotation.Nonnull; + +import java.util.Random; + +import static org.apache.flink.runtime.source.coordinator.CoordinatorTestUtils.waitForCoordinatorToProcessActions; + +/** The benchmark of watermark alignment. */ +public class SourceCoordinatorAlignmentBenchmark { + + /** The TimestampType determines how the timestamp or watermark is generated. */ + public enum TimestampType { + MONOTONE_INCREASING, + RANDOMIZE_MILLISECONDS + } + + private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L); + private static final Random RANDOM = new Random(); + + private SourceCoordinator<?, ?> sourceCoordinator; + private int numSubtasks; + private TimestampType timestampType; + + public SourceCoordinatorAlignmentBenchmark() {} + + public void setup(int numSubtasks, @Nonnull TimestampType timestampType) throws Exception { + SourceCoordinatorProvider<MockSourceSplit> provider = + new SourceCoordinatorProvider<>( + "SourceCoordinatorProviderTest", + OPERATOR_ID, + new MockSource(Boundedness.BOUNDED, 2), + 1, + new WatermarkAlignmentParams(1000L, "group1", Long.MAX_VALUE), + null); + this.sourceCoordinator = + (SourceCoordinator<?, ?>) + provider.getCoordinator( + new MockOperatorCoordinatorContext(OPERATOR_ID, numSubtasks)); + this.sourceCoordinator.start(); + this.numSubtasks = numSubtasks; + this.timestampType = timestampType; + + // Initialize the watermark for all subtasks. + sendReportedWatermarkToAllSubtasks(); + } + + public void teardown() throws Exception { + sourceCoordinator.close(); + } + + public void sendReportedWatermarkToAllSubtasks() { Review Comment: can you verify with some code profiler, that when calling this method repeatedly for a minute or so, what are you actually benchmarking? Where does the CPU spend most of its time? If those are the exact places that you would expect in production code, then all is good. However it shouldn't be things like `RANDOM.nextInt()` or `System.currentTimeMillis()` or other unrelated call. ########## flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorAlignmentBenchmark.java: ########## @@ -0,0 +1,97 @@ +/* + * 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.api.common.eventtime.WatermarkAlignmentParams; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.mocks.MockSource; +import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.source.event.ReportedWatermarkEvent; + +import javax.annotation.Nonnull; + +import java.util.Random; + +import static org.apache.flink.runtime.source.coordinator.CoordinatorTestUtils.waitForCoordinatorToProcessActions; + +/** The benchmark of watermark alignment. */ +public class SourceCoordinatorAlignmentBenchmark { + + /** The TimestampType determines how the timestamp or watermark is generated. */ + public enum TimestampType { + MONOTONE_INCREASING, + RANDOMIZE_MILLISECONDS + } + + private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L); + private static final Random RANDOM = new Random(); + + private SourceCoordinator<?, ?> sourceCoordinator; + private int numSubtasks; + private TimestampType timestampType; + + public SourceCoordinatorAlignmentBenchmark() {} + + public void setup(int numSubtasks, @Nonnull TimestampType timestampType) throws Exception { + SourceCoordinatorProvider<MockSourceSplit> provider = + new SourceCoordinatorProvider<>( + "SourceCoordinatorProviderTest", + OPERATOR_ID, + new MockSource(Boundedness.BOUNDED, 2), + 1, + new WatermarkAlignmentParams(1000L, "group1", Long.MAX_VALUE), + null); + this.sourceCoordinator = + (SourceCoordinator<?, ?>) + provider.getCoordinator( + new MockOperatorCoordinatorContext(OPERATOR_ID, numSubtasks)); + this.sourceCoordinator.start(); + this.numSubtasks = numSubtasks; + this.timestampType = timestampType; + + // Initialize the watermark for all subtasks. + sendReportedWatermarkToAllSubtasks(); + } + + public void teardown() throws Exception { + sourceCoordinator.close(); + } + + public void sendReportedWatermarkToAllSubtasks() { + for (int subtaskIndex = 0; subtaskIndex < numSubtasks; subtaskIndex++) { + sourceCoordinator.handleEventFromOperator( + subtaskIndex, 0, new ReportedWatermarkEvent(generateTimestamp())); + } + waitForCoordinatorToProcessActions(sourceCoordinator.getContext()); + } + + private long generateTimestamp() { + switch (timestampType) { + case MONOTONE_INCREASING: + return System.currentTimeMillis(); + case RANDOMIZE_MILLISECONDS: + return System.currentTimeMillis() / 1000 * 1000 + RANDOM.nextInt(1000); Review Comment: Don't you actually benchmark here `RANDOM.nextInt()` and `System.currentTimeMillis()` instead of the `SourceCoordinator`? 🤔 `Random` is a very heavy class with lot's of overheads. `ThreadLocalRandom` is better, but there is no good replacement of `System.currentTimeMillis`, which is also quite heavy (syscall). Usually the solution is have some pre-generated small array of timestamps in the `setup` method, that's later used in the benchmark. -- 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]
