Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

add async or sync for :ledger metadata index's rocksdb write #3368

Closed
Closed
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -50,22 +50,22 @@
*
* <p>The key is the ledgerId and the value is the {@link LedgerData} content.
*/
public class LedgerMetadataIndex implements Closeable {
public abstract class LedgerMetadataIndex implements Closeable {
// Non-ledger data should have negative ID
private static final long STORAGE_FLAGS = -0xeefd;
protected static final long STORAGE_FLAGS = -0xeefd;

// Contains all ledgers stored in the bookie
private final ConcurrentLongHashMap<LedgerData> ledgers;
private final AtomicInteger ledgersCount;

private final KeyValueStorage ledgersDb;
protected final KeyValueStorage ledgersDb;
private final LedgerMetadataIndexStats stats;

// Holds ledger modifications applied in memory map, and pending to be flushed on db
private final ConcurrentLinkedQueue<Entry<Long, LedgerData>> pendingLedgersUpdates;
protected final ConcurrentLinkedQueue<Entry<Long, LedgerData>> pendingLedgersUpdates;

// Holds ledger ids that were delete from memory map, and pending to be flushed on db
private final ConcurrentLinkedQueue<Long> pendingDeletedLedgers;
protected final ConcurrentLinkedQueue<Long> pendingDeletedLedgers;
private final ReentrantLock[] locks = new ReentrantLock[16];

public LedgerMetadataIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
Expand Down Expand Up @@ -302,43 +302,9 @@ public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException {
/**
* Flushes all pending changes.
*/
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();
}

public void removeDeletedLedgers() throws IOException {
LongWrapper key = LongWrapper.get();
public abstract void flush() throws IOException;

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();
}
public abstract void removeDeletedLedgers() throws IOException;

private ReentrantLock lockForLedger(long ledgerId) {
return locks[Math.abs((int) ledgerId) % locks.length];
Expand All @@ -363,32 +329,7 @@ int getStorageStateFlags() throws IOException {
}
}

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) {
int current = 0;
if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
current = (int) currentWrapper.getValue();
}
if (current == expected) {
ledgersDb.put(keyWrapper.array, newFlagsWrapper.array);
ledgersDb.sync();
return true;
}
}
} finally {
keyWrapper.recycle();
currentWrapper.recycle();
newFlagsWrapper.recycle();
}
return false;
}
abstract boolean setStorageStateFlags(int expected, int newFlags) throws IOException;

private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndex.class);

Expand All @@ -412,4 +353,11 @@ void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
}
}

public static LedgerMetadataIndex newInstance(ServerConfiguration conf, KeyValueStorageFactory storageFactory,
String basePath, StatsLogger stats) throws IOException {
if (!conf.getDbLedgerMetadataIndexSyncEnable()) {
return new LedgerMetadataIndexAsync(conf, storageFactory, basePath, stats);
}
return new LedgerMetadataIndexSync(conf, storageFactory, basePath, stats);
}
}
Original file line number Diff line number Diff line change
@@ -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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

int current = 0;
if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
current = (int) currentWrapper.getValue();
}
if (current == expected) {
ledgersDb.put(keyWrapper.array, newFlagsWrapper.array);
ledgersDb.sync();
return true;
}
}
} finally {
keyWrapper.recycle();
currentWrapper.recycle();
newFlagsWrapper.recycle();
}
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
/**
*
* 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>Synchronous write mode class,
* the key is the ledgerId and the value is the {@link LedgerData} content.
*/
public class LedgerMetadataIndexSync extends LedgerMetadataIndex {

private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndexSync.class);

public LedgerMetadataIndexSync(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();
KeyValueStorage.Batch batch = ledgersDb.newBatch();

try {
int updatedLedgers = 0;
while (!pendingLedgersUpdates.isEmpty()) {
Entry<Long, LedgerData> entry = pendingLedgersUpdates.poll();
key.set(entry.getKey());
byte[] value = entry.getValue().toByteArray();
batch.put(key.array, value);
++updatedLedgers;
}

if (log.isDebugEnabled()) {
log.debug("Persisting updates to {} ledgers", updatedLedgers);
}
} finally {
try {
batch.flush();
batch.clear();
} finally {
key.recycle();
batch.close();
}
}
}

@Override
public void removeDeletedLedgers() throws IOException {
LongWrapper key = LongWrapper.get();
KeyValueStorage.Batch batch = ledgersDb.newBatch();

try {
int deletedLedgers = 0;
while (!pendingDeletedLedgers.isEmpty()) {
long ledgerId = pendingDeletedLedgers.poll();
key.set(ledgerId);
batch.remove(key.array);
}

if (log.isDebugEnabled()) {
log.debug("Persisting deletes of ledgers {}", deletedLedgers);
}
} finally {
try {
batch.flush();
batch.clear();
} finally {
key.recycle();
batch.close();
}
}
}

@Override
boolean setStorageStateFlags(int expected, int newFlags) throws IOException {
LongWrapper keyWrapper = LongWrapper.get();
LongWrapper currentWrapper = LongWrapper.get();
LongWrapper newFlagsWrapper = LongWrapper.get();

KeyValueStorage.Batch batch = ledgersDb.newBatch();
try {
keyWrapper.set(STORAGE_FLAGS);
newFlagsWrapper.set(newFlags);
synchronized (ledgersDb) {
int current = 0;
if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
current = (int) currentWrapper.getValue();
}
if (current == expected) {
batch.put(keyWrapper.array, newFlagsWrapper.array);
return true;
}
}
} finally {
try {
batch.flush();
batch.clear();
} finally {
keyWrapper.recycle();
currentWrapper.recycle();
newFlagsWrapper.recycle();
batch.close();
}
}
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,8 @@ public boolean accept(long ledgerId) {

private Set<Long> getActiveLedgers(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath)
throws IOException {
LedgerMetadataIndex ledgers = new LedgerMetadataIndex(conf, storageFactory, basePath, NullStatsLogger.INSTANCE);
LedgerMetadataIndex ledgers = LedgerMetadataIndex.newInstance(conf,
storageFactory, basePath, NullStatsLogger.INSTANCE);
Set<Long> activeLedgers = Sets.newHashSet();
for (Long ledger : ledgers.getActiveLedgersInRange(0, Long.MAX_VALUE)) {
activeLedgers.add(ledger);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le

readCache = new ReadCache(allocator, readCacheMaxSize);

ledgerIndex = new LedgerMetadataIndex(conf,
ledgerIndex = LedgerMetadataIndex.newInstance(conf,
KeyValueStorageRocksDB.factory, indexBaseDir, ledgerIndexDirStatsLogger);
entryLocationIndex = new EntryLocationIndex(conf,
KeyValueStorageRocksDB.factory, indexBaseDir, ledgerIndexDirStatsLogger);
Expand Down
Loading