You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2022/07/31 10:35:57 UTC

[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3432: Tuning pool concurrency

hangc0276 commented on code in PR #3432:
URL: https://github.com/apache/bookkeeper/pull/3432#discussion_r933963487


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/DbLedgerStorageConfiguration.java:
##########
@@ -0,0 +1,161 @@
+/**
+ *
+ * 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.conf;
+
+// CHECKSTYLE.OFF: IllegalImport
+import com.google.common.annotations.VisibleForTesting;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.NettyRuntime;
+import io.netty.util.internal.PlatformDependent;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+// CHECKSTYLE.ON: IllegalImport
+
+public class DbLedgerStorageConfiguration extends ServerConfiguration {
+
+    private static final int MB = 1024 * 1024;
+
+    @VisibleForTesting
+    public static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb";
+
+    private static final long DEFAULT_WRITE_CACHE_MAX_SIZE_MB =
+            (long) (0.25 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    @VisibleForTesting
+    public static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb";
+
+    private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB =
+            (long) (0.25 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize";
+
+    private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100;
+
+    private static final String DIRECT_IO_ENTRYLOGGER = "dbStorage_directIOEntryLogger";
+
+    private static final String DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerTotalWriteBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB =
+            (long) (0.125 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerTotalReadBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB =
+            (long) (0.125 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerReadBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB = 8;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS =
+            "dbStorage_directIOEntryLoggerMaxFdCacheTimeSeconds";
+
+    private static final int DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS = 300;
+
+    @VisibleForTesting
+    public static final String MAX_THROTTLE_TIME_MILLIS = "dbStorage_maxThrottleTimeMs";
+
+    private static final long DEFAULT_MAX_THROTTLE_TIME_MILLIS = TimeUnit.SECONDS.toMillis(10);
+
+    public long getWriteCacheMaxSize() {
+        return getLongVariableOrDefault(WRITE_CACHE_MAX_SIZE_MB, DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB;
+    }
+
+    public long getReadCacheMaxSize() {
+        return getLongVariableOrDefault(READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB;
+    }
+
+    public int getReadAheadCacheBatchSize() {
+        return this.getInt(READ_AHEAD_CACHE_BATCH_SIZE, DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE);
+    }
+
+    public boolean isDirectIOEntryLoggerEnabled() {
+        return getBooleanVariableOrDefault(DIRECT_IO_ENTRYLOGGER, false);
+    }
+
+    public long getDirectIOEntryLoggerTotalWriteBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB,
+                DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB) * MB;
+    }
+
+    public long getDirectIOEntryLoggerTotalReadBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB,
+                DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB) * MB;
+    }
+
+    public long getDirectIOEntryLoggerReadBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB, DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB)
+                * MB;
+    }
+
+    public long getDirectIOEntryLoggerMaxFDCacheTimeSeconds() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS,
+                DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS);
+    }
+
+    public long getMaxThrottleTimeMillis() {
+        return this.getLong(MAX_THROTTLE_TIME_MILLIS, DEFAULT_MAX_THROTTLE_TIME_MILLIS);
+    }
+
+    /**
+     * The configured pooling concurrency for the allocator, if user config it, we should consider the unpooled direct
+     * memory which readCache and writeCache occupy when use DbLedgerStorage.
+     */
+    public int getAllocatorPoolingConcurrency() {
+        long writeCacheSize = this.getWriteCacheMaxSize();
+        long readCacheSize = this.getReadCacheMaxSize();
+        long availableDirectMemory = PlatformDependent.maxDirectMemory() - writeCacheSize - readCacheSize;

Review Comment:
   Please use `PlatformDependent.estimateMaxDirectMemory()` instead of `PlatformDependent.maxDirectMemory()`. Please refer to: https://github.com/apache/bookkeeper/pull/2989



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/DbLedgerStorageConfiguration.java:
##########
@@ -0,0 +1,161 @@
+/**
+ *
+ * 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.conf;
+
+// CHECKSTYLE.OFF: IllegalImport
+import com.google.common.annotations.VisibleForTesting;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.NettyRuntime;
+import io.netty.util.internal.PlatformDependent;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+// CHECKSTYLE.ON: IllegalImport
+
+public class DbLedgerStorageConfiguration extends ServerConfiguration {
+
+    private static final int MB = 1024 * 1024;
+
+    @VisibleForTesting
+    public static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb";
+
+    private static final long DEFAULT_WRITE_CACHE_MAX_SIZE_MB =
+            (long) (0.25 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    @VisibleForTesting
+    public static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb";
+
+    private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB =
+            (long) (0.25 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize";
+
+    private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100;
+
+    private static final String DIRECT_IO_ENTRYLOGGER = "dbStorage_directIOEntryLogger";
+
+    private static final String DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerTotalWriteBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB =
+            (long) (0.125 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerTotalReadBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB =
+            (long) (0.125 * PlatformDependent.estimateMaxDirectMemory()) / MB;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB =
+            "dbStorage_directIOEntryLoggerReadBufferSizeMb";
+
+    private static final long DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB = 8;
+
+    private static final String DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS =
+            "dbStorage_directIOEntryLoggerMaxFdCacheTimeSeconds";
+
+    private static final int DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS = 300;
+
+    @VisibleForTesting
+    public static final String MAX_THROTTLE_TIME_MILLIS = "dbStorage_maxThrottleTimeMs";
+
+    private static final long DEFAULT_MAX_THROTTLE_TIME_MILLIS = TimeUnit.SECONDS.toMillis(10);
+
+    public long getWriteCacheMaxSize() {
+        return getLongVariableOrDefault(WRITE_CACHE_MAX_SIZE_MB, DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB;
+    }
+
+    public long getReadCacheMaxSize() {
+        return getLongVariableOrDefault(READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB;
+    }
+
+    public int getReadAheadCacheBatchSize() {
+        return this.getInt(READ_AHEAD_CACHE_BATCH_SIZE, DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE);
+    }
+
+    public boolean isDirectIOEntryLoggerEnabled() {
+        return getBooleanVariableOrDefault(DIRECT_IO_ENTRYLOGGER, false);
+    }
+
+    public long getDirectIOEntryLoggerTotalWriteBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB,
+                DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB) * MB;
+    }
+
+    public long getDirectIOEntryLoggerTotalReadBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB,
+                DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB) * MB;
+    }
+
+    public long getDirectIOEntryLoggerReadBufferSize() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB, DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB)
+                * MB;
+    }
+
+    public long getDirectIOEntryLoggerMaxFDCacheTimeSeconds() {
+        return getLongVariableOrDefault(DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS,
+                DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS);
+    }
+
+    public long getMaxThrottleTimeMillis() {
+        return this.getLong(MAX_THROTTLE_TIME_MILLIS, DEFAULT_MAX_THROTTLE_TIME_MILLIS);
+    }
+
+    /**
+     * The configured pooling concurrency for the allocator, if user config it, we should consider the unpooled direct
+     * memory which readCache and writeCache occupy when use DbLedgerStorage.
+     */
+    public int getAllocatorPoolingConcurrency() {
+        long writeCacheSize = this.getWriteCacheMaxSize();
+        long readCacheSize = this.getReadCacheMaxSize();
+        long availableDirectMemory = PlatformDependent.maxDirectMemory() - writeCacheSize - readCacheSize;

Review Comment:
   In dbLedgerStorage, 
   - If we use `DefaultEntryLogger`, only the whole write-cache will be allocated at one time in the init state, and the read-cache will be allocated on-demand.
   - If we use `DirectEnryLogger`, both the write-cache and the read-cache are allocated on-demand.
   
   So I think we'd better keep the default implementation.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java:
##########
@@ -146,19 +113,18 @@ public class DbLedgerStorage implements LedgerStorage {
     public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager,
                            LedgerDirsManager indexDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator)
             throws IOException {
-        long writeCacheMaxSize = getLongVariableOrDefault(conf, WRITE_CACHE_MAX_SIZE_MB,
-                DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB;
-        long readCacheMaxSize = getLongVariableOrDefault(conf, READ_AHEAD_CACHE_MAX_SIZE_MB,
-                DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB;
-        boolean directIOEntryLogger = getBooleanVariableOrDefault(conf, DIRECT_IO_ENTRYLOGGER, false);
+        DbLedgerStorageConfiguration dbLedgerStorageConf = conf.toDbLedgerStorageConfiguration();
+        long writeCacheMaxSize = dbLedgerStorageConf.getWriteCacheMaxSize();
+        long readCacheMaxSize = dbLedgerStorageConf.getReadCacheMaxSize();
+        boolean directIOEntryLogger = dbLedgerStorageConf.isDirectIOEntryLoggerEnabled();
 
         this.allocator = allocator;
         this.numberOfDirs = ledgerDirsManager.getAllLedgerDirs().size();
 
         log.info("Started Db Ledger Storage");
         log.info(" - Number of directories: {}", numberOfDirs);
-        log.info(" - Write cache size: {} MB", writeCacheMaxSize / MB);
-        log.info(" - Read Cache: {} MB", readCacheMaxSize / MB);
+        log.info(" - Write cache size: {} MB", writeCacheMaxSize / 1024 / 1024);

Review Comment:
   Could we define `MB` ?



-- 
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@bookkeeper.apache.org

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