Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/2727

Change subject: [NO ISSUE][STO] Limit BufferCacheHeaderHelper Pool Size
......................................................................

[NO ISSUE][STO] Limit BufferCacheHeaderHelper Pool Size

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Limit the BufferCacheHeaderHelper pool size to the IO
  requests queue size.

Change-Id: I7914241110e9fe3e5e21f35cf03ba6ecdf132343
---
M 
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
M 
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
2 files changed, 7 insertions(+), 4 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb 
refs/changes/27/2727/1

diff --git 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 1f845bf..d13ca8d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -55,8 +55,8 @@
     /*
      * Constants
      */
+    public static final int IO_REQUEST_QUEUE_SIZE = 100; // TODO: Make 
configurable
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final int IO_REQUEST_QUEUE_SIZE = 100; // TODO: Make 
configurable
     private static final String WORKSPACE_FILE_SUFFIX = ".waf";
     private static final FilenameFilter WORKSPACE_FILES_FILTER = (dir, name) 
-> name.endsWith(WORKSPACE_FILE_SUFFIX);
     /*
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index f8facf5..5d8a701 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.storage.common.buffercache;
 
+import static org.apache.hyracks.control.nc.io.IOManager.IO_REQUEST_QUEUE_SIZE;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -27,9 +29,9 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
@@ -74,7 +76,8 @@
     private final CleanerThread cleanerThread;
     private final Map<Integer, BufferedFileHandle> fileInfoMap;
     private final AsyncFIFOPageQueueManager fifoWriter;
-    private final Queue<BufferCacheHeaderHelper> headerPageCache = new 
ConcurrentLinkedQueue<>();
+    private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache =
+            new ArrayBlockingQueue<>(IO_REQUEST_QUEUE_SIZE);
 
     //DEBUG
     private static final Level fileOpsLevel = Level.DEBUG;

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2727
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I7914241110e9fe3e5e21f35cf03ba6ecdf132343
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <[email protected]>

Reply via email to