JunRuiLee commented on code in PR #25552: URL: https://github.com/apache/flink/pull/25552#discussion_r1898365716
########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/SubpartitionSlice.java: ########## @@ -0,0 +1,109 @@ +/* + * 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.scheduler.adaptivebatch.util; + +import org.apache.flink.runtime.executiongraph.IndexRange; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.IntStream; + +/** Helper class that provides information for subpartition slice. */ Review Comment: this javadoc does not have enough information ########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BlockingInputInfo.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.scheduler.adaptivebatch; + +import org.apache.flink.runtime.executiongraph.IndexRange; +import org.apache.flink.runtime.executiongraph.ResultPartitionBytes; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkState; + +public class BlockingInputInfo implements BlockingResultInfo { + private final BlockingResultInfo blockingResultInfo; + private final int inputTypeNumber; + private final boolean existInterInputsKeyCorrelation; + private final boolean existIntraInputKeyCorrelation; + + public BlockingInputInfo( + BlockingResultInfo blockingResultInfo, + int inputTypeNumber, + boolean existInterInputsKeyCorrelation, + boolean existIntraInputKeyCorrelation) { + this.blockingResultInfo = blockingResultInfo; Review Comment: checkNotNull ########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BlockingInputInfo.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.scheduler.adaptivebatch; + +import org.apache.flink.runtime.executiongraph.IndexRange; +import org.apache.flink.runtime.executiongraph.ResultPartitionBytes; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkState; + +public class BlockingInputInfo implements BlockingResultInfo { + private final BlockingResultInfo blockingResultInfo; + private final int inputTypeNumber; + private final boolean existInterInputsKeyCorrelation; + private final boolean existIntraInputKeyCorrelation; + + public BlockingInputInfo( + BlockingResultInfo blockingResultInfo, + int inputTypeNumber, + boolean existInterInputsKeyCorrelation, + boolean existIntraInputKeyCorrelation) { + this.blockingResultInfo = blockingResultInfo; + this.inputTypeNumber = inputTypeNumber; + this.existInterInputsKeyCorrelation = existInterInputsKeyCorrelation; + this.existIntraInputKeyCorrelation = existIntraInputKeyCorrelation; + } + + public BlockingResultInfo getConsumedResultInfo() { + return blockingResultInfo; + } + + public int getInputTypeNumber() { + return inputTypeNumber; + } + + public boolean existIntraInputKeyCorrelation() { + return existIntraInputKeyCorrelation; + } + + public boolean existInterInputsKeyCorrelation() { + return existInterInputsKeyCorrelation; + } + + public List<Long> getAggregatedSubpartitionBytes() { + if (blockingResultInfo instanceof AllToAllBlockingResultInfo) { + return ((AllToAllBlockingResultInfo) blockingResultInfo) + .getAggregatedSubpartitionBytes(); + } + return Collections.emptyList(); Review Comment: Do we expect this method should be call although the result info is point-wise? If not, we can do some check here ########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BlockingInputInfo.java: ########## @@ -45,10 +45,6 @@ public BlockingInputInfo( this.existIntraInputKeyCorrelation = existIntraInputKeyCorrelation; } - public BlockingResultInfo getConsumedResultInfo() { Review Comment: this update does not belong in this commit. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org