You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2016/04/05 08:50:43 UTC

bookkeeper git commit: BOOKKEEPER-578: LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory

Repository: bookkeeper
Updated Branches:
  refs/heads/master b1c12c0f4 -> d607b366b


BOOKKEEPER-578: LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory

Author: Matteo Merli <mm...@apache.org>

Reviewers: Sijie Guo <si...@apache.org>

Closes #28 from merlimat/bk-578


Project: http://git-wip-us.apache.org/repos/asf/bookkeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/bookkeeper/commit/d607b366
Tree: http://git-wip-us.apache.org/repos/asf/bookkeeper/tree/d607b366
Diff: http://git-wip-us.apache.org/repos/asf/bookkeeper/diff/d607b366

Branch: refs/heads/master
Commit: d607b366ba493617408888a768a99ee3f3ffb976
Parents: b1c12c0
Author: Matteo Merli <mm...@apache.org>
Authored: Mon Apr 4 23:50:37 2016 -0700
Committer: Sijie Guo <si...@apache.org>
Committed: Mon Apr 4 23:50:37 2016 -0700

----------------------------------------------------------------------
 .../bookkeeper/bookie/IndexInMemPageMgr.java    | 11 ++--
 .../bookkeeper/util/DirectMemoryUtils.java      | 57 ++++++++++++++++++++
 2 files changed, 64 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/d607b366/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java
index 454a383..30eaf72 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java
@@ -23,6 +23,7 @@ package org.apache.bookkeeper.bookie;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.DirectMemoryUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -332,14 +333,16 @@ class IndexInMemPageMgr {
         this.indexPersistenceManager = indexPersistenceManager;
         this.pageMapAndList = new InMemPageCollection();
 
+        long maxDirectMemory = DirectMemoryUtils.maxDirectMemory();
+
         if (conf.getPageLimit() <= 0) {
-            // allocate half of the memory to the page cache
-            this.pageLimit = (int) ((Runtime.getRuntime().maxMemory() / 3) / this.pageSize);
+            // By default, allocate a third of the direct memory to the page cache
+            this.pageLimit = (int) ((maxDirectMemory / 3) / this.pageSize);
         } else {
             this.pageLimit = conf.getPageLimit();
         }
-        LOG.info("maxMemory = {}, pageSize = {}, pageLimit = {}", new Object[] { Runtime.getRuntime().maxMemory(),
-                        pageSize, pageLimit });
+        LOG.info("maxDirectMemory = {}, pageSize = {}, pageLimit = {}",
+                new Object[] { maxDirectMemory, pageSize, pageLimit });
         // Expose Stats
         statsLogger.registerGauge(NUM_INDEX_PAGES, new Gauge<Number>() {
             @Override

http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/d607b366/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DirectMemoryUtils.java
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DirectMemoryUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DirectMemoryUtils.java
new file mode 100644
index 0000000..9f82bb2
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DirectMemoryUtils.java
@@ -0,0 +1,57 @@
+/*
+ *
+ * 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.bookkeeper.util;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.lang.reflect.Method;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DirectMemoryUtils {
+    /**
+     * Returns the max configured size of direct memory for the JVM process.
+     *
+     * Direct memory can be specified with the flag <code>-XX:MaxDirectMemorySize=8G</code> on the command line. If not
+     * specified, the default value will be set to the max size of the JVM heap.
+     */
+    public static long maxDirectMemory() {
+        try {
+
+            Class<?> VM = Class.forName("sun.misc.VM");
+            Method maxDirectMemory = VM.getDeclaredMethod("maxDirectMemory");
+            Object result = maxDirectMemory.invoke(null, (Object[]) null);
+
+            checkNotNull(result);
+            checkArgument(result instanceof Long);
+            return (Long) result;
+        } catch (Exception e) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Failed to get maxDirectMemory size from sun.misc.VM, falling back to max heap size", e);
+            }
+            return Runtime.getRuntime().maxMemory();
+        }
+    }
+
+    private static final Logger LOG = LoggerFactory.getLogger(DirectMemoryUtils.class);
+}