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 all 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 @@ -24,7 +24,6 @@
import com.google.common.collect.Iterables;
import com.google.protobuf.ByteString;
import io.netty.buffer.ByteBuf;

import java.io.Closeable;
import java.io.IOException;
import java.util.AbstractMap.SimpleEntry;
Expand All @@ -33,7 +32,6 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;

import org.apache.bookkeeper.bookie.Bookie;
import org.apache.bookkeeper.bookie.BookieException;
import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData;
Expand All @@ -50,22 +48,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,93 +300,32 @@ 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 abstract void flush() throws IOException;

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

private ReentrantLock lockForLedger(long ledgerId) {
return locks[Math.abs((int) ledgerId) % locks.length];
}

int getStorageStateFlags() throws IOException {
synchronized int getStorageStateFlags() throws IOException {
LongWrapper keyWrapper = LongWrapper.get();
LongWrapper currentWrapper = LongWrapper.get();

try {
keyWrapper.set(STORAGE_FLAGS);
synchronized (ledgersDb) {
int current = 0;
if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
current = (int) currentWrapper.getValue();
}
return current;
int current = 0;
if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
current = (int) currentWrapper.getValue();
}
return current;
} finally {
keyWrapper.recycle();
currentWrapper.recycle();
}
}

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 +349,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,114 @@
/**
*
* 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
synchronized 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);
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;
}
}
Loading