You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/29 08:00:44 UTC

[GitHub] [flink] reswqa opened a new pull request, #20100: [FLINK-27905] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

reswqa opened a new pull request, #20100:
URL: https://github.com/apache/flink/pull/20100

   ## What is the purpose of the change
   
   *Introduce `HsSpillingStrategy` and Implement `HsSelectiveSpillingStrategy` to support make a spilling decision for hybrid shuffle.*
   
   ## Brief change log
   
     - *Introduce `HsSpillingStrategy` interface for all spilling strategy.*
     - *Introduce `HsSelectiveSpillingStrategy` to reduce disk writes as much as possible.*
   
   
   ## Verifying this change
   
   Introduce `HsSelectiveSpillingStrategyTest`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive):  no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r920001944


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategyTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSelectiveSpillingStrategy}. */
+class HsSelectiveSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final float SELECTIVE_SPILL_THRESHOLD = 0.7f;
+
+    public static final float SELECTIVE_SPILL_BUFFER_RATIO = 0.4f;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsSelectiveSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setSelectiveStrategySpillThreshold(SELECTIVE_SPILL_THRESHOLD)
+                                .setSelectiveStrategySpillBufferRatio(SELECTIVE_SPILL_BUFFER_RATIO)
+                                .build());
+    }
+
+    @Test
+    void testOnBufferFinished() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action finishedAction =
+                spillStrategy.onBufferFinished(spillInfoProvider, bufferWithIdentity);
+        assertThat(finishedAction.isImmediate()).isTrue();
+        assertThat(finishedAction.getBufferToRelease()).isEmpty();
+        assertThat(finishedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnBufferConsumed() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action consumedAction =
+                spillStrategy.onBufferConsumed(spillInfoProvider, bufferWithIdentity);
+        assertThat(consumedAction.isImmediate()).isTrue();
+        assertThat(consumedAction.getBufferToRelease()).hasSize(1);
+        assertThat(consumedAction.getBufferToRelease()).element(0).isEqualTo(bufferWithIdentity);
+        assertThat(consumedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnUsedMemoryChanged() {
+        testOnUsedMemoryLow();
+        testOnUsedMemoryHigh();
+    }
+
+    void testOnUsedMemoryLow() {
+        final int bufferPoolSize = 10;
+        final int bufferThreshold = (int) (bufferPoolSize * SELECTIVE_SPILL_THRESHOLD);
+        spillInfoProvider.setCurrentPoolSizeSupplier(() -> bufferPoolSize);
+        spillInfoProvider.setNumTotalUnSpilledBuffersSupplier(() -> bufferThreshold - 1);
+        Action action = spillStrategy.onUsedMemoryChanged(spillInfoProvider);
+        assertThat(action.isImmediate()).isTrue();
+        assertThat(action.getBufferToSpill()).isEmpty();
+        assertThat(action.getBufferToRelease()).isEmpty();
+    }
+
+    void testOnUsedMemoryHigh() {
+        final int subpartition1 = 0;
+        final int subpartition2 = 1;
+        final int subpartition3 = 2;
+
+        Map<Integer, Deque<BufferWithIdentity>> subpartitionBuffers = new HashMap<>();
+        subpartitionBuffers.put(subpartition1, createBufferWithIdentities(subpartition1, 0, 4, 6));
+        subpartitionBuffers.put(subpartition2, createBufferWithIdentities(subpartition2, 3, 7, 9));
+        subpartitionBuffers.put(subpartition3, createBufferWithIdentities(subpartition3, 5, 6, 11));
+
+        spillInfoProvider.setUnSpilledBuffersFunction(subpartitionBuffers::get);
+        spillInfoProvider.setConsumptionProgressSupplier(() -> Arrays.asList(-1, 1, 2));
+
+        final int totalBuffers = 9;
+        final int expectedNumSpilled = (int) (totalBuffers * SELECTIVE_SPILL_BUFFER_RATIO);

Review Comment:
   Yes, it should be more reasonable to use `poolSize` instead of `totalBufferSize`, I will also modify the calculation logic in `HsSelectiveSpillingStrategy`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong closed pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
xintongsong closed pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation
URL: https://github.com/apache/flink/pull/20100


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
xintongsong commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r917627889


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    /** The proportion of buffers to be spilled. */
+    public static final float SPILL_BUFFER_RATIO = 0.4f;
+
+    /**
+     * When the number of buffers that have been requested exceeds this threshold, trigger the
+     * spilling operation.
+     */
+    public static final float SPILL_THRESHOLD = 0.7f;

Review Comment:
   These can be included into `HybridShuffleConfiguration`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Component which encapsulates the spilling logic. It is responsible for deciding when and which
+ * buffers will be spilled to disk and released from memory.
+ */
+public interface HsSpillingStrategy {
+    /**
+     * Callback for {@link Buffer} instances when it becomes finished.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param finishedBuffer info of finished buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer);
+
+    /**
+     * Callback for {@link Buffer} instances when it is consumed by downstream task.
+     *
+     * <p>Note that only this method is called by downstream thread, everything else is the current
+     * task main thread.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param consumedBuffer info of consumed buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer);
+
+    /**
+     * Decide whether the current situation will trigger an {@link Action} and return it.
+     *
+     * @param spillInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action maybeSpill(HsSpillingInfoProvider spillInfoProvider);
+
+    class Action {
+        /** A collection of buffer that needs to be spilled to disk. */
+        private final List<BufferWithIdentity> bufferToSpill;
+
+        /** A collection of buffer that needs to be released. */
+        private final List<BufferWithIdentity> bufferToRelease;
+
+        public static final Action EMPTY =
+                new Action(Collections.emptyList(), Collections.emptyList());
+
+        public Action(
+                List<BufferWithIdentity> bufferToSpill, List<BufferWithIdentity> bufferToRelease) {
+            this.bufferToSpill = bufferToSpill;
+            this.bufferToRelease = bufferToRelease;
+        }
+
+        public List<BufferWithIdentity> getBufferToSpill() {
+            return bufferToSpill;
+        }
+
+        public List<BufferWithIdentity> getBufferToRelease() {
+            return bufferToRelease;
+        }
+
+        public static Action onlyNeedSpill(List<BufferWithIdentity> bufferWithIdentityList) {
+            return new Action(bufferWithIdentityList, Collections.emptyList());
+        }
+
+        public static Action onlyNeedRelease(List<BufferWithIdentity> bufferWithIdentityList) {
+            return new Action(Collections.emptyList(), bufferWithIdentityList);
+        }

Review Comment:
   We may use a builder here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    /** The proportion of buffers to be spilled. */
+    public static final float SPILL_BUFFER_RATIO = 0.4f;
+
+    /**
+     * When the number of buffers that have been requested exceeds this threshold, trigger the
+     * spilling operation.
+     */
+    public static final float SPILL_THRESHOLD = 0.7f;
+
+    /**
+     * For the case of buffer finished, there is no need to make a decision for {@link
+     * HsSelectiveSpillingStrategy}.
+     */
+    @Override
+    public Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer) {
+        return Action.EMPTY;
+    }
+
+    /**
+     * For the case of buffer consumed, this buffer need release. The control of the buffer is taken
+     * over by the downstream task.
+     */
+    @Override
+    public Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer) {
+        return Action.onlyNeedRelease(Collections.singletonList(consumedBuffer));
+    }
+
+    /**
+     * When the amount of memory used exceeds the threshold, trigger the spilling operation and
+     * score the buffer of each subpartition. The lower the score, the more likely the buffer will
+     * be consumed in the next time, and should be kept in memory as much as possible. Select all
+     * buffers that need to be spilled according to the score from high to low.
+     */
+    @Override
+    public Action maybeSpill(HsSpillingInfoProvider spillInfoProvider) {
+        if (spillInfoProvider.getNumRequestedBuffers()
+                < spillInfoProvider.getPoolSize() * SPILL_THRESHOLD) {
+            return Action.EMPTY;
+        }
+
+        List<Integer> consumptionProgress = spillInfoProvider.getConsumptionProgress();
+        PriorityQueue<BufferIdentityWithScore> heap = new PriorityQueue<>();
+        int numBuffers = 0;
+        for (int channel = 0; channel < spillInfoProvider.getNumSubpartitions(); channel++) {
+            Collection<BufferWithIdentity> finishedBuffers =
+                    spillInfoProvider.getSubpartitionBuffers(channel);
+            int subpartitionProgress = consumptionProgress.get(channel);
+            // compute score for each subpartition buffers.
+            for (BufferWithIdentity bufferWithInfo : finishedBuffers) {
+                // buffer index in finishedBuffers must be greater than or equal to the downstream
+                // consumption progress, so the score must be greater than or equal to 0.
+                int score = bufferWithInfo.getBufferIndex() - subpartitionProgress;
+                heap.add(new BufferIdentityWithScore(bufferWithInfo, score));
+                numBuffers++;
+            }
+        }
+        List<BufferWithIdentity> toSpill = new ArrayList<>();
+        int spillNum = (int) (numBuffers * SPILL_BUFFER_RATIO);
+        for (int i = 0; i < spillNum; i++) {
+            toSpill.add(Preconditions.checkNotNull(heap.poll()).bufferWithInfo);
+        }

Review Comment:
   Not sure about calculating the score for all the buffers. Consider we have a list/queue of buffers for each subpartition, it's probably good enough to always compare the first element of each queue (once the buffer is dequeued, reorder w.r.t. the new first element).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsAllSpillingStrategy.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** A special implementation of {@link HsSpillingStrategy} that spilled all buffer to disk. */
+public class HsAllSpillingStrategy implements HsSpillingStrategy {
+
+    public static final int NUM_BUFFERS_TRIGGER_SPILLED = 10;

Review Comment:
   This can be included in `HybridShuffleConfiguration`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Component which encapsulates the spilling logic. It is responsible for deciding when and which
+ * buffers will be spilled to disk and released from memory.
+ */
+public interface HsSpillingStrategy {
+    /**
+     * Callback for {@link Buffer} instances when it becomes finished.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param finishedBuffer info of finished buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer);
+
+    /**
+     * Callback for {@link Buffer} instances when it is consumed by downstream task.
+     *
+     * <p>Note that only this method is called by downstream thread, everything else is the current
+     * task main thread.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param consumedBuffer info of consumed buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer);
+
+    /**
+     * Decide whether the current situation will trigger an {@link Action} and return it.
+     *
+     * @param spillInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action maybeSpill(HsSpillingInfoProvider spillInfoProvider);

Review Comment:
   ```suggestion
       Action checkForAction(HsSpillingInfoProvider spillInfoProvider);
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsAllSpillingStrategy.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** A special implementation of {@link HsSpillingStrategy} that spilled all buffer to disk. */
+public class HsAllSpillingStrategy implements HsSpillingStrategy {
+
+    public static final int NUM_BUFFERS_TRIGGER_SPILLED = 10;
+
+    public int unSpilledFinishedBuffers;

Review Comment:
   It is preferred not to maintain this inside the strategy. We may instead provide this via `HsSpillingInfoProvider`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919997634


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyTestBase.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/** Test base for {@link HsSpillingStrategy}. */
+abstract class HsSpillingStrategyTestBase {
+    public static final int NUM_SUBPARTITIONS = 3;
+
+    public static final int MEMORY_SEGMENT_SIZE = 128;
+
+    protected HsSpillingStrategy spillStrategy;
+
+    protected TestingSpillingInfoProvider spillInfoProvider;
+
+    @BeforeEach
+    void setup() {
+        this.spillInfoProvider = new TestingSpillingInfoProvider(NUM_SUBPARTITIONS);
+    }
+
+    @Test
+    abstract void testOnBufferFinished();
+
+    @Test
+    abstract void testOnBufferConsumed();
+
+    @Test
+    abstract void testOnUsedMemoryChanged();

Review Comment:
   removed and refactor this class into `HsSpillingStrategyTestUtils`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919959209


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/** Mock {@link HsSpillingInfoProvider} for test. */
+public class TestingSpillingInfoProvider implements HsSpillingInfoProvider {
+    private final int numSubpartitions;
+
+    private Supplier<List<Integer>> consumptionProgressSupplier = ArrayList::new;
+
+    private Function<Integer, Deque<BufferWithIdentity>> unSpilledBuffersFunction =
+            (ignore) -> new ArrayDeque<>();
+
+    private Supplier<Integer> numTotalUnSpilledBuffersSupplier = () -> 0;
+
+    private Supplier<Integer> currentPoolSizeSupplier = () -> 0;

Review Comment:
   changed to create by builder



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919998839


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {

Review Comment:
   remove lifecycle methods, and make `HsSpillingStrategy` become class field.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r917675487


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Component which encapsulates the spilling logic. It is responsible for deciding when and which
+ * buffers will be spilled to disk and released from memory.
+ */
+public interface HsSpillingStrategy {
+    /**
+     * Callback for {@link Buffer} instances when it becomes finished.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param finishedBuffer info of finished buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer);
+
+    /**
+     * Callback for {@link Buffer} instances when it is consumed by downstream task.
+     *
+     * <p>Note that only this method is called by downstream thread, everything else is the current
+     * task main thread.
+     *
+     * @param spillingInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @param consumedBuffer info of consumed buffer.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer);
+
+    /**
+     * Decide whether the current situation will trigger an {@link Action} and return it.
+     *
+     * @param spillInfoProvider to compute which buffers need to be spilled into disk and which
+     *     buffers need be released from memory.
+     * @return Which buffers need to be spilled and which buffers need to be released.
+     */
+    Action maybeSpill(HsSpillingInfoProvider spillInfoProvider);

Review Comment:
   renamed



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918596608


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    public final float spillBufferRatio;
+
+    public final float spillThreshold;
+
+    public HsSelectiveSpillingStrategy(HybridShuffleConfiguration hybridShuffleConfiguration) {
+        spillThreshold = hybridShuffleConfiguration.getSelectiveSpillThreshold();
+        spillBufferRatio = hybridShuffleConfiguration.getSelectiveSpillBufferRatio();
+    }
+
+    /**
+     * For the case of buffer finished, there is no need to make a decision for {@link
+     * HsSelectiveSpillingStrategy}.
+     */
+    @Override
+    public Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer) {
+        return Action.EMPTY;
+    }
+
+    /**
+     * For the case of buffer consumed, this buffer need release. The control of the buffer is taken
+     * over by the downstream task.
+     */
+    @Override
+    public Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer) {
+        return Action.builder().addBufferToRelease(consumedBuffer).build();
+    }
+
+    /**
+     * When the amount of memory used exceeds the threshold, trigger the spilling operation and
+     * score the buffer of each subpartition. The lower the score, the more likely the buffer will
+     * be consumed in the next time, and should be kept in memory as much as possible. Select all
+     * buffers that need to be spilled according to the score from high to low.
+     */
+    @Override
+    public Action checkForAction(HsSpillingInfoProvider spillInfoProvider) {
+        if (spillInfoProvider.getNumTotalUnSpilledBuffers()
+                < spillInfoProvider.getPoolSize() * spillThreshold) {
+            return Action.EMPTY;
+        }
+
+        List<Integer> consumptionProgress = spillInfoProvider.getConsumptionProgress();
+        PriorityQueue<BufferScoreQueue> heap = new PriorityQueue<>();
+        for (int channel = 0; channel < spillInfoProvider.getNumSubpartitions(); channel++) {
+            Deque<BufferWithIdentity> unSpilledBuffers =
+                    spillInfoProvider.getUnSpilledBuffers(channel);
+            // In order to prevent the impact on the original queue, copy it and then pass to heap.
+            Deque<BufferWithIdentity> copiedQueue = new ArrayDeque<>(unSpilledBuffers);

Review Comment:
   Rewritten here using `PeekingIterator`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r920000157


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsFullSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setFullStrategyNumBuffersTriggerSpilling(
+                                        NUM_BUFFERS_TRIGGER_SPILLING)
+                                .build());
+    }
+
+    @Override
+    @Test
+    void testOnBufferFinished() {
+        testUnSpillBufferBelowThreshold();
+        testUnSpilledBufferEqualThreshold();

Review Comment:
   Splits these into single methods.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918596608


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    public final float spillBufferRatio;
+
+    public final float spillThreshold;
+
+    public HsSelectiveSpillingStrategy(HybridShuffleConfiguration hybridShuffleConfiguration) {
+        spillThreshold = hybridShuffleConfiguration.getSelectiveSpillThreshold();
+        spillBufferRatio = hybridShuffleConfiguration.getSelectiveSpillBufferRatio();
+    }
+
+    /**
+     * For the case of buffer finished, there is no need to make a decision for {@link
+     * HsSelectiveSpillingStrategy}.
+     */
+    @Override
+    public Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer) {
+        return Action.EMPTY;
+    }
+
+    /**
+     * For the case of buffer consumed, this buffer need release. The control of the buffer is taken
+     * over by the downstream task.
+     */
+    @Override
+    public Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer) {
+        return Action.builder().addBufferToRelease(consumedBuffer).build();
+    }
+
+    /**
+     * When the amount of memory used exceeds the threshold, trigger the spilling operation and
+     * score the buffer of each subpartition. The lower the score, the more likely the buffer will
+     * be consumed in the next time, and should be kept in memory as much as possible. Select all
+     * buffers that need to be spilled according to the score from high to low.
+     */
+    @Override
+    public Action checkForAction(HsSpillingInfoProvider spillInfoProvider) {
+        if (spillInfoProvider.getNumTotalUnSpilledBuffers()
+                < spillInfoProvider.getPoolSize() * spillThreshold) {
+            return Action.EMPTY;
+        }
+
+        List<Integer> consumptionProgress = spillInfoProvider.getConsumptionProgress();
+        PriorityQueue<BufferScoreQueue> heap = new PriorityQueue<>();
+        for (int channel = 0; channel < spillInfoProvider.getNumSubpartitions(); channel++) {
+            Deque<BufferWithIdentity> unSpilledBuffers =
+                    spillInfoProvider.getUnSpilledBuffers(channel);
+            // In order to prevent the impact on the original queue, copy it and then pass to heap.
+            Deque<BufferWithIdentity> copiedQueue = new ArrayDeque<>(unSpilledBuffers);

Review Comment:
   Rewritten here using PeekingIterator



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918600252


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HybridShuffleConfiguration.java:
##########
@@ -74,4 +88,82 @@ public int getMaxBuffersReadAhead() {
     public Duration getBufferRequestTimeout() {
         return bufferRequestTimeout;
     }
+
+    /**
+     * When the number of buffers that have been requested exceeds this threshold, trigger the
+     * spilling operation. Used by {@link HsSelectiveSpillingStrategy}.
+     */
+    public float getSelectiveSpillThreshold() {
+        return selectiveSpillThreshold;
+    }
+
+    /** The proportion of buffers to be spilled. Used by {@link HsSelectiveSpillingStrategy}. */
+    public float getSelectiveSpillBufferRatio() {
+        return selectiveSpillBufferRatio;
+    }
+
+    /**
+     * When the number of unSpilled buffers equal to this value, trigger the spilling operation.
+     * Used by {@link HsAllSpillingStrategy}.
+     */
+    public int getNumBuffersTriggerSpilling() {
+        return numBuffersTriggerSpilling;
+    }

Review Comment:
   renamed



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918178186


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.PriorityQueue;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    /** The proportion of buffers to be spilled. */
+    public static final float SPILL_BUFFER_RATIO = 0.4f;
+
+    /**
+     * When the number of buffers that have been requested exceeds this threshold, trigger the
+     * spilling operation.
+     */
+    public static final float SPILL_THRESHOLD = 0.7f;
+
+    /**
+     * For the case of buffer finished, there is no need to make a decision for {@link
+     * HsSelectiveSpillingStrategy}.
+     */
+    @Override
+    public Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer) {
+        return Action.EMPTY;
+    }
+
+    /**
+     * For the case of buffer consumed, this buffer need release. The control of the buffer is taken
+     * over by the downstream task.
+     */
+    @Override
+    public Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer) {
+        return Action.onlyNeedRelease(Collections.singletonList(consumedBuffer));
+    }
+
+    /**
+     * When the amount of memory used exceeds the threshold, trigger the spilling operation and
+     * score the buffer of each subpartition. The lower the score, the more likely the buffer will
+     * be consumed in the next time, and should be kept in memory as much as possible. Select all
+     * buffers that need to be spilled according to the score from high to low.
+     */
+    @Override
+    public Action maybeSpill(HsSpillingInfoProvider spillInfoProvider) {
+        if (spillInfoProvider.getNumRequestedBuffers()
+                < spillInfoProvider.getPoolSize() * SPILL_THRESHOLD) {
+            return Action.EMPTY;
+        }
+
+        List<Integer> consumptionProgress = spillInfoProvider.getConsumptionProgress();
+        PriorityQueue<BufferIdentityWithScore> heap = new PriorityQueue<>();
+        int numBuffers = 0;
+        for (int channel = 0; channel < spillInfoProvider.getNumSubpartitions(); channel++) {
+            Collection<BufferWithIdentity> finishedBuffers =
+                    spillInfoProvider.getSubpartitionBuffers(channel);
+            int subpartitionProgress = consumptionProgress.get(channel);
+            // compute score for each subpartition buffers.
+            for (BufferWithIdentity bufferWithInfo : finishedBuffers) {
+                // buffer index in finishedBuffers must be greater than or equal to the downstream
+                // consumption progress, so the score must be greater than or equal to 0.
+                int score = bufferWithInfo.getBufferIndex() - subpartitionProgress;
+                heap.add(new BufferIdentityWithScore(bufferWithInfo, score));
+                numBuffers++;
+            }
+        }
+        List<BufferWithIdentity> toSpill = new ArrayList<>();
+        int spillNum = (int) (numBuffers * SPILL_BUFFER_RATIO);
+        for (int i = 0; i < spillNum; i++) {
+            toSpill.add(Preconditions.checkNotNull(heap.poll()).bufferWithInfo);
+        }

Review Comment:
   It's really a nice idea, this can greatly improve the efficiency of the algorithm when there are a large number of buffers.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
xintongsong commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918520765


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingInfoProvider.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.Deque;
+import java.util.List;
+
+/** This component is responsible for providing some information needed for the spill decision. */
+public interface HsSpillingInfoProvider {
+    /**
+     * Get the number of downstream consumers.
+     *
+     * @return Number of subpartitions.
+     */
+    int getNumSubpartitions();
+
+    /**
+     * Get all downstream consumption progress.
+     *
+     * @return A list containing all downstream consumption progress, if the downstream subpartition
+     *     view has not been registered, the corresponding return value is -1.
+     */
+    List<Integer> getConsumptionProgress();

Review Comment:
   "Consuming progress" is ambiguous. It can be "last consumed buffer index", or "next buffer index to consume".



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsAllSpillingStrategy.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** A special implementation of {@link HsSpillingStrategy} that spilled all buffer to disk. */
+public class HsAllSpillingStrategy implements HsSpillingStrategy {
+    private final int numBuffersTriggerSpilling;
+
+    public HsAllSpillingStrategy(HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.numBuffersTriggerSpilling = hybridShuffleConfiguration.getNumBuffersTriggerSpilling();
+    }
+
+    /**
+     * For the case of buffer finished, whenever the unSpilledBuffers reaches {@link
+     * #numBuffersTriggerSpilling}, spill and release all buffers.
+     */

Review Comment:
   This should be a comment, not a JavaDoc.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HybridShuffleConfiguration.java:
##########
@@ -74,4 +88,82 @@ public int getMaxBuffersReadAhead() {
     public Duration getBufferRequestTimeout() {
         return bufferRequestTimeout;
     }
+
+    /**
+     * When the number of buffers that have been requested exceeds this threshold, trigger the
+     * spilling operation. Used by {@link HsSelectiveSpillingStrategy}.
+     */
+    public float getSelectiveSpillThreshold() {
+        return selectiveSpillThreshold;
+    }
+
+    /** The proportion of buffers to be spilled. Used by {@link HsSelectiveSpillingStrategy}. */
+    public float getSelectiveSpillBufferRatio() {
+        return selectiveSpillBufferRatio;
+    }
+
+    /**
+     * When the number of unSpilled buffers equal to this value, trigger the spilling operation.
+     * Used by {@link HsAllSpillingStrategy}.
+     */
+    public int getNumBuffersTriggerSpilling() {
+        return numBuffersTriggerSpilling;
+    }

Review Comment:
   I'd suggest to prefix these properties with "Selective|FullStrategy", in clearly indicate that the configuration is used by a specific strategy.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategy.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A special implementation of {@link HsSpillingStrategy} that reduce disk writes as much as
+ * possible.
+ */
+public class HsSelectiveSpillingStrategy implements HsSpillingStrategy {
+    public final float spillBufferRatio;
+
+    public final float spillThreshold;
+
+    public HsSelectiveSpillingStrategy(HybridShuffleConfiguration hybridShuffleConfiguration) {
+        spillThreshold = hybridShuffleConfiguration.getSelectiveSpillThreshold();
+        spillBufferRatio = hybridShuffleConfiguration.getSelectiveSpillBufferRatio();
+    }
+
+    /**
+     * For the case of buffer finished, there is no need to make a decision for {@link
+     * HsSelectiveSpillingStrategy}.
+     */
+    @Override
+    public Action onBufferFinished(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity finishedBuffer) {
+        return Action.EMPTY;
+    }
+
+    /**
+     * For the case of buffer consumed, this buffer need release. The control of the buffer is taken
+     * over by the downstream task.
+     */
+    @Override
+    public Action onBufferConsumed(
+            HsSpillingInfoProvider spillingInfoProvider, BufferWithIdentity consumedBuffer) {
+        return Action.builder().addBufferToRelease(consumedBuffer).build();
+    }
+
+    /**
+     * When the amount of memory used exceeds the threshold, trigger the spilling operation and
+     * score the buffer of each subpartition. The lower the score, the more likely the buffer will
+     * be consumed in the next time, and should be kept in memory as much as possible. Select all
+     * buffers that need to be spilled according to the score from high to low.
+     */
+    @Override
+    public Action checkForAction(HsSpillingInfoProvider spillInfoProvider) {
+        if (spillInfoProvider.getNumTotalUnSpilledBuffers()
+                < spillInfoProvider.getPoolSize() * spillThreshold) {
+            return Action.EMPTY;
+        }
+
+        List<Integer> consumptionProgress = spillInfoProvider.getConsumptionProgress();
+        PriorityQueue<BufferScoreQueue> heap = new PriorityQueue<>();
+        for (int channel = 0; channel < spillInfoProvider.getNumSubpartitions(); channel++) {
+            Deque<BufferWithIdentity> unSpilledBuffers =
+                    spillInfoProvider.getUnSpilledBuffers(channel);
+            // In order to prevent the impact on the original queue, copy it and then pass to heap.
+            Deque<BufferWithIdentity> copiedQueue = new ArrayDeque<>(unSpilledBuffers);

Review Comment:
   Instead of copying the deque, we may consider using an iterator.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919998839


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {

Review Comment:
   remove lifecycle methods, and make HsSpillingStrategy become class field.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r920000157


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsFullSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setFullStrategyNumBuffersTriggerSpilling(
+                                        NUM_BUFFERS_TRIGGER_SPILLING)
+                                .build());
+    }
+
+    @Override
+    @Test
+    void testOnBufferFinished() {
+        testUnSpillBufferBelowThreshold();
+        testUnSpilledBufferEqualThreshold();

Review Comment:
   Splits these into multiple single methods.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on PR #20100:
URL: https://github.com/apache/flink/pull/20100#issuecomment-1183161247

   Thanks @xintongsong, I have resolved all comments, PTAL~


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
xintongsong commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919831119


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/TestingSpillingInfoProvider.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/** Mock {@link HsSpillingInfoProvider} for test. */
+public class TestingSpillingInfoProvider implements HsSpillingInfoProvider {
+    private final int numSubpartitions;
+
+    private Supplier<List<Integer>> consumptionProgressSupplier = ArrayList::new;
+
+    private Function<Integer, Deque<BufferWithIdentity>> unSpilledBuffersFunction =
+            (ignore) -> new ArrayDeque<>();
+
+    private Supplier<Integer> numTotalUnSpilledBuffersSupplier = () -> 0;
+
+    private Supplier<Integer> currentPoolSizeSupplier = () -> 0;

Review Comment:
   These should be final. They can be decided via a builder/factory. Once created, behaviors of the testing provider should be deterministic. 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyTestBase.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/** Test base for {@link HsSpillingStrategy}. */
+abstract class HsSpillingStrategyTestBase {
+    public static final int NUM_SUBPARTITIONS = 3;
+
+    public static final int MEMORY_SEGMENT_SIZE = 128;
+
+    protected HsSpillingStrategy spillStrategy;
+
+    protected TestingSpillingInfoProvider spillInfoProvider;
+
+    @BeforeEach
+    void setup() {
+        this.spillInfoProvider = new TestingSpillingInfoProvider(NUM_SUBPARTITIONS);
+    }
+
+    @Test
+    abstract void testOnBufferFinished();
+
+    @Test
+    abstract void testOnBufferConsumed();
+
+    @Test
+    abstract void testOnUsedMemoryChanged();

Review Comment:
   Why do we need these abstract methods?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {

Review Comment:
   Why do we need to reconstruct the strategy for each test?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFullSpillingStrategyTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsFullSpillingStrategy}. */
+class HsFullSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final int NUM_BUFFERS_TRIGGER_SPILLING = 10;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsFullSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setFullStrategyNumBuffersTriggerSpilling(
+                                        NUM_BUFFERS_TRIGGER_SPILLING)
+                                .build());
+    }
+
+    @Override
+    @Test
+    void testOnBufferFinished() {
+        testUnSpillBufferBelowThreshold();
+        testUnSpilledBufferEqualThreshold();

Review Comment:
   It is not recommended to combine multiple test cases.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategyTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSelectiveSpillingStrategy}. */
+class HsSelectiveSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final float SELECTIVE_SPILL_THRESHOLD = 0.7f;
+
+    public static final float SELECTIVE_SPILL_BUFFER_RATIO = 0.4f;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsSelectiveSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setSelectiveStrategySpillThreshold(SELECTIVE_SPILL_THRESHOLD)
+                                .setSelectiveStrategySpillBufferRatio(SELECTIVE_SPILL_BUFFER_RATIO)
+                                .build());
+    }
+
+    @Test
+    void testOnBufferFinished() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action finishedAction =
+                spillStrategy.onBufferFinished(spillInfoProvider, bufferWithIdentity);
+        assertThat(finishedAction.isImmediate()).isTrue();
+        assertThat(finishedAction.getBufferToRelease()).isEmpty();
+        assertThat(finishedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnBufferConsumed() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action consumedAction =
+                spillStrategy.onBufferConsumed(spillInfoProvider, bufferWithIdentity);
+        assertThat(consumedAction.isImmediate()).isTrue();
+        assertThat(consumedAction.getBufferToRelease()).hasSize(1);
+        assertThat(consumedAction.getBufferToRelease()).element(0).isEqualTo(bufferWithIdentity);
+        assertThat(consumedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnUsedMemoryChanged() {
+        testOnUsedMemoryLow();
+        testOnUsedMemoryHigh();
+    }
+
+    void testOnUsedMemoryLow() {
+        final int bufferPoolSize = 10;
+        final int bufferThreshold = (int) (bufferPoolSize * SELECTIVE_SPILL_THRESHOLD);
+        spillInfoProvider.setCurrentPoolSizeSupplier(() -> bufferPoolSize);
+        spillInfoProvider.setNumTotalUnSpilledBuffersSupplier(() -> bufferThreshold - 1);
+        Action action = spillStrategy.onUsedMemoryChanged(spillInfoProvider);
+        assertThat(action.isImmediate()).isTrue();
+        assertThat(action.getBufferToSpill()).isEmpty();
+        assertThat(action.getBufferToRelease()).isEmpty();
+    }
+
+    void testOnUsedMemoryHigh() {
+        final int subpartition1 = 0;
+        final int subpartition2 = 1;
+        final int subpartition3 = 2;
+
+        Map<Integer, Deque<BufferWithIdentity>> subpartitionBuffers = new HashMap<>();
+        subpartitionBuffers.put(subpartition1, createBufferWithIdentities(subpartition1, 0, 4, 6));
+        subpartitionBuffers.put(subpartition2, createBufferWithIdentities(subpartition2, 3, 7, 9));
+        subpartitionBuffers.put(subpartition3, createBufferWithIdentities(subpartition3, 5, 6, 11));
+
+        spillInfoProvider.setUnSpilledBuffersFunction(subpartitionBuffers::get);
+        spillInfoProvider.setConsumptionProgressSupplier(() -> Arrays.asList(-1, 1, 2));

Review Comment:
   It's not easy to tell which buffer is expected to be spilled with higher priority.
   
   I'd suggest to set the consuming progress with constants:
   ```
   final int progress1 = 10;
   final int progress2 = 20;
   final int progress3 = 30;
   ```
   
   And set the buffer index relative to the consuming progress:
   ```
   createBufferWithIdentities(subpartition1, progress1 + 0, progress1 + 3, progress1 + 6);
   createBufferWithIdentities(subpartition2, progress2 + 1, progress2 + 4, progress2 + 7);
   createBufferWithIdentities(subpartition3, progress3 + 2, progress3 + 5, progress3 + 8);
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategyTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSelectiveSpillingStrategy}. */
+class HsSelectiveSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final float SELECTIVE_SPILL_THRESHOLD = 0.7f;
+
+    public static final float SELECTIVE_SPILL_BUFFER_RATIO = 0.4f;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsSelectiveSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setSelectiveStrategySpillThreshold(SELECTIVE_SPILL_THRESHOLD)
+                                .setSelectiveStrategySpillBufferRatio(SELECTIVE_SPILL_BUFFER_RATIO)
+                                .build());
+    }
+
+    @Test
+    void testOnBufferFinished() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action finishedAction =
+                spillStrategy.onBufferFinished(spillInfoProvider, bufferWithIdentity);
+        assertThat(finishedAction.isImmediate()).isTrue();
+        assertThat(finishedAction.getBufferToRelease()).isEmpty();
+        assertThat(finishedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnBufferConsumed() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action consumedAction =
+                spillStrategy.onBufferConsumed(spillInfoProvider, bufferWithIdentity);
+        assertThat(consumedAction.isImmediate()).isTrue();
+        assertThat(consumedAction.getBufferToRelease()).hasSize(1);
+        assertThat(consumedAction.getBufferToRelease()).element(0).isEqualTo(bufferWithIdentity);
+        assertThat(consumedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnUsedMemoryChanged() {
+        testOnUsedMemoryLow();
+        testOnUsedMemoryHigh();
+    }
+
+    void testOnUsedMemoryLow() {
+        final int bufferPoolSize = 10;
+        final int bufferThreshold = (int) (bufferPoolSize * SELECTIVE_SPILL_THRESHOLD);
+        spillInfoProvider.setCurrentPoolSizeSupplier(() -> bufferPoolSize);
+        spillInfoProvider.setNumTotalUnSpilledBuffersSupplier(() -> bufferThreshold - 1);
+        Action action = spillStrategy.onUsedMemoryChanged(spillInfoProvider);
+        assertThat(action.isImmediate()).isTrue();
+        assertThat(action.getBufferToSpill()).isEmpty();
+        assertThat(action.getBufferToRelease()).isEmpty();
+    }
+
+    void testOnUsedMemoryHigh() {
+        final int subpartition1 = 0;
+        final int subpartition2 = 1;
+        final int subpartition3 = 2;
+
+        Map<Integer, Deque<BufferWithIdentity>> subpartitionBuffers = new HashMap<>();
+        subpartitionBuffers.put(subpartition1, createBufferWithIdentities(subpartition1, 0, 4, 6));
+        subpartitionBuffers.put(subpartition2, createBufferWithIdentities(subpartition2, 3, 7, 9));
+        subpartitionBuffers.put(subpartition3, createBufferWithIdentities(subpartition3, 5, 6, 11));
+
+        spillInfoProvider.setUnSpilledBuffersFunction(subpartitionBuffers::get);
+        spillInfoProvider.setConsumptionProgressSupplier(() -> Arrays.asList(-1, 1, 2));

Review Comment:
   Then you won't need `getBufferAndScores` and can directly tell which buffers are expected to be spilled.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategyTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSelectiveSpillingStrategy}. */
+class HsSelectiveSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final float SELECTIVE_SPILL_THRESHOLD = 0.7f;
+
+    public static final float SELECTIVE_SPILL_BUFFER_RATIO = 0.4f;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsSelectiveSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setSelectiveStrategySpillThreshold(SELECTIVE_SPILL_THRESHOLD)
+                                .setSelectiveStrategySpillBufferRatio(SELECTIVE_SPILL_BUFFER_RATIO)
+                                .build());
+    }
+
+    @Test
+    void testOnBufferFinished() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action finishedAction =
+                spillStrategy.onBufferFinished(spillInfoProvider, bufferWithIdentity);
+        assertThat(finishedAction.isImmediate()).isTrue();
+        assertThat(finishedAction.getBufferToRelease()).isEmpty();
+        assertThat(finishedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnBufferConsumed() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action consumedAction =
+                spillStrategy.onBufferConsumed(spillInfoProvider, bufferWithIdentity);
+        assertThat(consumedAction.isImmediate()).isTrue();
+        assertThat(consumedAction.getBufferToRelease()).hasSize(1);
+        assertThat(consumedAction.getBufferToRelease()).element(0).isEqualTo(bufferWithIdentity);
+        assertThat(consumedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnUsedMemoryChanged() {
+        testOnUsedMemoryLow();
+        testOnUsedMemoryHigh();
+    }
+
+    void testOnUsedMemoryLow() {
+        final int bufferPoolSize = 10;
+        final int bufferThreshold = (int) (bufferPoolSize * SELECTIVE_SPILL_THRESHOLD);
+        spillInfoProvider.setCurrentPoolSizeSupplier(() -> bufferPoolSize);
+        spillInfoProvider.setNumTotalUnSpilledBuffersSupplier(() -> bufferThreshold - 1);
+        Action action = spillStrategy.onUsedMemoryChanged(spillInfoProvider);
+        assertThat(action.isImmediate()).isTrue();
+        assertThat(action.getBufferToSpill()).isEmpty();
+        assertThat(action.getBufferToRelease()).isEmpty();
+    }
+
+    void testOnUsedMemoryHigh() {
+        final int subpartition1 = 0;
+        final int subpartition2 = 1;
+        final int subpartition3 = 2;
+
+        Map<Integer, Deque<BufferWithIdentity>> subpartitionBuffers = new HashMap<>();
+        subpartitionBuffers.put(subpartition1, createBufferWithIdentities(subpartition1, 0, 4, 6));
+        subpartitionBuffers.put(subpartition2, createBufferWithIdentities(subpartition2, 3, 7, 9));
+        subpartitionBuffers.put(subpartition3, createBufferWithIdentities(subpartition3, 5, 6, 11));
+
+        spillInfoProvider.setUnSpilledBuffersFunction(subpartitionBuffers::get);
+        spillInfoProvider.setConsumptionProgressSupplier(() -> Arrays.asList(-1, 1, 2));
+
+        final int totalBuffers = 9;
+        final int expectedNumSpilled = (int) (totalBuffers * SELECTIVE_SPILL_BUFFER_RATIO);

Review Comment:
   Shouldn't this be `poolSize * SELECTIVE_SPILL_BUFFER_RATIO`?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r918587520


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingInfoProvider.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import java.util.Deque;
+import java.util.List;
+
+/** This component is responsible for providing some information needed for the spill decision. */
+public interface HsSpillingInfoProvider {
+    /**
+     * Get the number of downstream consumers.
+     *
+     * @return Number of subpartitions.
+     */
+    int getNumSubpartitions();
+
+    /**
+     * Get all downstream consumption progress.
+     *
+     * @return A list containing all downstream consumption progress, if the downstream subpartition
+     *     view has not been registered, the corresponding return value is -1.
+     */
+    List<Integer> getConsumptionProgress();

Review Comment:
   renamed to `getNextBufferIndexToConsume`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r920014506


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSelectiveSpillingStrategyTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Action;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.NeedLockAction;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSelectiveSpillingStrategy}. */
+class HsSelectiveSpillingStrategyTest extends HsSpillingStrategyTestBase {
+    public static final float SELECTIVE_SPILL_THRESHOLD = 0.7f;
+
+    public static final float SELECTIVE_SPILL_BUFFER_RATIO = 0.4f;
+
+    @BeforeEach
+    void before() {
+        this.spillStrategy =
+                new HsSelectiveSpillingStrategy(
+                        HybridShuffleConfiguration.builder(NUM_SUBPARTITIONS, 1)
+                                .setSelectiveStrategySpillThreshold(SELECTIVE_SPILL_THRESHOLD)
+                                .setSelectiveStrategySpillBufferRatio(SELECTIVE_SPILL_BUFFER_RATIO)
+                                .build());
+    }
+
+    @Test
+    void testOnBufferFinished() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action finishedAction =
+                spillStrategy.onBufferFinished(spillInfoProvider, bufferWithIdentity);
+        assertThat(finishedAction.isImmediate()).isTrue();
+        assertThat(finishedAction.getBufferToRelease()).isEmpty();
+        assertThat(finishedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnBufferConsumed() {
+        BufferWithIdentity bufferWithIdentity = new BufferWithIdentity(createBuffer(), 0, 0);
+        Action consumedAction =
+                spillStrategy.onBufferConsumed(spillInfoProvider, bufferWithIdentity);
+        assertThat(consumedAction.isImmediate()).isTrue();
+        assertThat(consumedAction.getBufferToRelease()).hasSize(1);
+        assertThat(consumedAction.getBufferToRelease()).element(0).isEqualTo(bufferWithIdentity);
+        assertThat(consumedAction.getBufferToSpill()).isEmpty();
+    }
+
+    @Test
+    void testOnUsedMemoryChanged() {
+        testOnUsedMemoryLow();
+        testOnUsedMemoryHigh();
+    }
+
+    void testOnUsedMemoryLow() {
+        final int bufferPoolSize = 10;
+        final int bufferThreshold = (int) (bufferPoolSize * SELECTIVE_SPILL_THRESHOLD);
+        spillInfoProvider.setCurrentPoolSizeSupplier(() -> bufferPoolSize);
+        spillInfoProvider.setNumTotalUnSpilledBuffersSupplier(() -> bufferThreshold - 1);
+        Action action = spillStrategy.onUsedMemoryChanged(spillInfoProvider);
+        assertThat(action.isImmediate()).isTrue();
+        assertThat(action.getBufferToSpill()).isEmpty();
+        assertThat(action.getBufferToRelease()).isEmpty();
+    }
+
+    void testOnUsedMemoryHigh() {
+        final int subpartition1 = 0;
+        final int subpartition2 = 1;
+        final int subpartition3 = 2;
+
+        Map<Integer, Deque<BufferWithIdentity>> subpartitionBuffers = new HashMap<>();
+        subpartitionBuffers.put(subpartition1, createBufferWithIdentities(subpartition1, 0, 4, 6));
+        subpartitionBuffers.put(subpartition2, createBufferWithIdentities(subpartition2, 3, 7, 9));
+        subpartitionBuffers.put(subpartition3, createBufferWithIdentities(subpartition3, 5, 6, 11));
+
+        spillInfoProvider.setUnSpilledBuffersFunction(subpartitionBuffers::get);
+        spillInfoProvider.setConsumptionProgressSupplier(() -> Arrays.asList(-1, 1, 2));

Review Comment:
   Good idea! Changed to this way.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20100: [FLINK-27905] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20100:
URL: https://github.com/apache/flink/pull/20100#issuecomment-1169664598

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "71434177b679cfcd805030e10cd6bc0c6e54d6a5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "71434177b679cfcd805030e10cd6bc0c6e54d6a5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 71434177b679cfcd805030e10cd6bc0c6e54d6a5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20100: [FLINK-27905][runtime] Introduce HsSpillingStrategy and HsSelectiveSpillingStrategy implementation

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20100:
URL: https://github.com/apache/flink/pull/20100#discussion_r919997634


##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyTestBase.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io.network.partition.hybrid;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/** Test base for {@link HsSpillingStrategy}. */
+abstract class HsSpillingStrategyTestBase {
+    public static final int NUM_SUBPARTITIONS = 3;
+
+    public static final int MEMORY_SEGMENT_SIZE = 128;
+
+    protected HsSpillingStrategy spillStrategy;
+
+    protected TestingSpillingInfoProvider spillInfoProvider;
+
+    @BeforeEach
+    void setup() {
+        this.spillInfoProvider = new TestingSpillingInfoProvider(NUM_SUBPARTITIONS);
+    }
+
+    @Test
+    abstract void testOnBufferFinished();
+
+    @Test
+    abstract void testOnBufferConsumed();
+
+    @Test
+    abstract void testOnUsedMemoryChanged();

Review Comment:
   removed abstract methods and refactor this class into `HsSpillingStrategyTestUtils`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org