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/07/11 08:02:57 UTC

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

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