You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2022/08/20 04:56:03 UTC

[GitHub] [bookkeeper] StevenLuMT opened a new pull request, #3368: add async or sync for :ledger metadata index's rocksdb write

StevenLuMT opened a new pull request, #3368:
URL: https://github.com/apache/bookkeeper/pull/3368

   Descriptions of the changes in this PR:
   
   ### Motivation
   
   bookie flush mainly composed of three pieces:
         1.  flush-entrylog:   it's flushing entrylog 
         2. flush-locations-index: it's flushing entrly location index,  use **sync** mode to flush
         3. flush-ledger-index:  it's  flushing ledger metadata index, this index(LedgerMetadataIndex) use **async** mode to flush
    
   sync is different from async:
   - sync mode:  
         1.  create a batch; 
         5.  add msg to the batch
         6.  call method(batch.flush) to flush the batch
   
   - async mode:  
         1. just call method(sync)  to write the data
         2. the rocksdb server will be timed to flush the data 
   
   ### Changes
   
   1. add async or sync for : ledger metadata index rocksdb write
   2. add switch to open this feature, default not open, use default async 


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

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


[GitHub] [bookkeeper] StevenLuMT commented on a diff in pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on code in PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#discussion_r949975958


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexAsync.java:
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * 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.bookie.storage.ldb;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maintains an index for the ledgers metadata.
+ *
+ * <p>Asynchronous write mode class,
+ * the key is the ledgerId and the value is the {@link LedgerData} content.
+ */
+public class LedgerMetadataIndexAsync extends LedgerMetadataIndex {
+
+    private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndexAsync.class);
+
+    public LedgerMetadataIndexAsync(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
+                                    StatsLogger stats) throws IOException {
+        super(conf, storageFactory, basePath, stats);
+    }
+
+    /**
+     * Flushes all pending changes.
+     */
+    @Override
+    public void flush() throws IOException {
+        LongWrapper key = LongWrapper.get();
+
+        int updatedLedgers = 0;
+        while (!pendingLedgersUpdates.isEmpty()) {
+            Entry<Long, LedgerData> entry = pendingLedgersUpdates.poll();
+            key.set(entry.getKey());
+            byte[] value = entry.getValue().toByteArray();
+            ledgersDb.put(key.array, value);
+            ++updatedLedgers;
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Persisting updates to {} ledgers", updatedLedgers);
+        }
+
+        ledgersDb.sync();
+        key.recycle();
+    }
+
+    @Override
+    public void removeDeletedLedgers() throws IOException {
+        LongWrapper key = LongWrapper.get();
+
+        int deletedLedgers = 0;
+        while (!pendingDeletedLedgers.isEmpty()) {
+            long ledgerId = pendingDeletedLedgers.poll();
+            key.set(ledgerId);
+            ledgersDb.delete(key.array);
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Persisting deletes of ledgers {}", deletedLedgers);
+        }
+
+        ledgersDb.sync();
+        key.recycle();
+    }
+
+    @Override
+    boolean setStorageStateFlags(int expected, int newFlags) throws IOException {
+        LongWrapper keyWrapper = LongWrapper.get();
+        LongWrapper currentWrapper = LongWrapper.get();
+        LongWrapper newFlagsWrapper = LongWrapper.get();
+
+        try {
+            keyWrapper.set(STORAGE_FLAGS);
+            newFlagsWrapper.set(newFlags);
+            synchronized (ledgersDb) {

Review Comment:
   This logic has not been updated, and the code to move according to #2936 
   It doesn't seem to be a problem, or if I modify synchronized to the method, I will make a unified correction to the historical code. @eolivelli 



-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1166917650

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1167092935

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1168997321

   rerun failure checks
   


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1167264502

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1221437483

   rerun failure checks


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

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


[GitHub] [bookkeeper] StevenLuMT closed pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT closed pull request #3368: add async or sync for :ledger metadata index's rocksdb write
URL: https://github.com/apache/bookkeeper/pull/3368


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

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


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#discussion_r949895963


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexAsync.java:
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * 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.bookie.storage.ldb;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maintains an index for the ledgers metadata.
+ *
+ * <p>Asynchronous write mode class,
+ * the key is the ledgerId and the value is the {@link LedgerData} content.
+ */
+public class LedgerMetadataIndexAsync extends LedgerMetadataIndex {
+
+    private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndexAsync.class);
+
+    public LedgerMetadataIndexAsync(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
+                                    StatsLogger stats) throws IOException {
+        super(conf, storageFactory, basePath, stats);
+    }
+
+    /**
+     * Flushes all pending changes.
+     */
+    @Override
+    public void flush() throws IOException {
+        LongWrapper key = LongWrapper.get();
+
+        int updatedLedgers = 0;
+        while (!pendingLedgersUpdates.isEmpty()) {
+            Entry<Long, LedgerData> entry = pendingLedgersUpdates.poll();
+            key.set(entry.getKey());
+            byte[] value = entry.getValue().toByteArray();
+            ledgersDb.put(key.array, value);
+            ++updatedLedgers;
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Persisting updates to {} ledgers", updatedLedgers);
+        }
+
+        ledgersDb.sync();
+        key.recycle();
+    }
+
+    @Override
+    public void removeDeletedLedgers() throws IOException {
+        LongWrapper key = LongWrapper.get();
+
+        int deletedLedgers = 0;
+        while (!pendingDeletedLedgers.isEmpty()) {
+            long ledgerId = pendingDeletedLedgers.poll();
+            key.set(ledgerId);
+            ledgersDb.delete(key.array);
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Persisting deletes of ledgers {}", deletedLedgers);
+        }
+
+        ledgersDb.sync();
+        key.recycle();
+    }
+
+    @Override
+    boolean setStorageStateFlags(int expected, int newFlags) throws IOException {
+        LongWrapper keyWrapper = LongWrapper.get();
+        LongWrapper currentWrapper = LongWrapper.get();
+        LongWrapper newFlagsWrapper = LongWrapper.get();
+
+        try {
+            keyWrapper.set(STORAGE_FLAGS);
+            newFlagsWrapper.set(newFlags);
+            synchronized (ledgersDb) {

Review Comment:
   it seems to be that synchronisation around  `ledgersDb` is not consistent.
   sometimes we don't access the variable that way.
   
   I wonder why spotbugs does not complain



-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1166847082

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1167305780

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1167218638

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3368: add async or sync for :ledger metadata index's rocksdb write

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3368:
URL: https://github.com/apache/bookkeeper/pull/3368#issuecomment-1221232982

   reopen the pr, run the new workflow


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

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