You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by ni...@apache.org on 2022/04/06 08:13:19 UTC

[bookkeeper] branch master updated: Use netty maxDirectMemory instead of DirectMemoryUtils

This is an automated email from the ASF dual-hosted git repository.

nicoloboschi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new d4eba6295 Use netty maxDirectMemory instead of DirectMemoryUtils
d4eba6295 is described below

commit d4eba629504930ca6179dee220e075b489c193a7
Author: ZhangJian He <sh...@gmail.com>
AuthorDate: Wed Apr 6 16:13:13 2022 +0800

    Use netty maxDirectMemory instead of DirectMemoryUtils
    
    ### Motivation
    
    Our `DirectMemoryUtils` has huge limit, it can't work well with other jvm. The Netty `PlatformDependent.maxDirectMemory();` is more generic.
    
    ### Changes
    Use `PlatformDependent.maxDirectMemory();` instead of `DirectMemoryUtils`
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Andrey Yegorov <None>, Matteo Merli <mm...@apache.org>, Nicolò Boschi <bo...@gmail.com>
    
    This closes #2989 from Shoothzj/direct-memory
---
 .../bookkeeper/bookie/IndexInMemPageMgr.java       |  6 ++-
 .../apache/bookkeeper/util/DirectMemoryUtils.java  | 60 ----------------------
 2 files changed, 4 insertions(+), 62 deletions(-)

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 2a360205e..cb75d7e71 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
@@ -27,6 +27,9 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_RE
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.NUM_INDEX_PAGES;
 
 import com.google.common.base.Stopwatch;
+//CHECKSTYLE.OFF: IllegalImport
+import io.netty.util.internal.PlatformDependent;
+//CHECKSTYLE.ON: IllegalImport
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -47,7 +50,6 @@ import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.DirectMemoryUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -350,7 +352,7 @@ class IndexInMemPageMgr {
         this.indexPersistenceManager = indexPersistenceManager;
         this.pageMapAndList = new InMemPageCollection(statsLogger);
 
-        long maxDirectMemory = DirectMemoryUtils.maxDirectMemory();
+        long maxDirectMemory = PlatformDependent.estimateMaxDirectMemory();
 
         if (conf.getPageLimit() <= 0) {
             // By default, allocate a third of the direct memory to the page cache
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
deleted file mode 100644
index 8a5362ade..000000000
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DirectMemoryUtils.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- *
- * 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;
-
-/**
- * DirectMemory Utility.
- */
-public class DirectMemoryUtils {
-    /**
-     * Returns the max configured size of direct memory for the JVM process.
-     *
-     * <p>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);
-}