TanYuxin-tyx commented on code in PR #22352:
URL: https://github.com/apache/flink/pull/22352#discussion_r1193828115


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImpl.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.tiered.storage;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.LocalBufferPool;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+
+import 
org.apache.flink.shaded.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation for {@link TieredStorageMemoryManager}. This is to 
request or recycle buffers
+ * from {@link LocalBufferPool} for different memory owners, for example, the 
tiers, the buffer
+ * accumulator, etc.
+ *
+ * <p>Note that the memory owner should register its {@link 
TieredStorageMemorySpec} firstly before
+ * requesting buffers.
+ */
+public class TieredStorageMemoryManagerImpl implements 
TieredStorageMemoryManager {
+
+    /** Initial delay before checking flush. */
+    public static final int DEFAULT_CHECK_FLUSH_INITIAL_DELAY_MS = 10;
+
+    /** Check flush period. */
+    public static final int DEFAULT_CHECK_FLUSH_PERIOD_DURATION_MS = 50;
+
+    /** The tiered storage memory specs of each memory user owner. */
+    private final Map<Object, TieredStorageMemorySpec> tieredMemorySpecs;
+
+    /** The registered callbacks to flush the buffers in the registered tiered 
storages. */
+    private final List<Runnable> bufferFlushCallbacks;
+
+    /** The buffer pool usage ratio of triggering the registered storages to 
flush buffers. */
+    private final float numBuffersTriggerFlushRatio;
+
+    /**
+     * Indicate whether to start the buffer flush checker thread. If the 
memory manager is used in
+     * downstream, the field will be false because no buffer flush checker is 
needed.
+     */
+    private final boolean shouldStartBufferFlushChecker;

Review Comment:
   I believe the thread belongs to this class. But the thread name and the 
switch name are not that right. I renamed the `boolean` to 
`needPeriodicalCheckReclaimBuffer` and renamed `numTriggerReclaimBuffersRatio`.
   
   The accuracy of the memory usage ratio may be compromised due to the varying 
buffer pool sizes. However, this only impacts a single iteration of the buffer 
usage check. Upon the next iteration, the buffer reclaim will eventually be 
triggered.
   
   The Java doc also added the above reason descriptions of adding the thread.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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

Reply via email to