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

Managed ledger uses ReadHandle in read path #1513

Merged
merged 5 commits into from
Apr 10, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -24,7 +24,7 @@
import com.google.common.base.Charsets;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.client.BookKeeper.DigestType;
import org.apache.bookkeeper.client.api.DigestType;

/**
* Configuration class for a ManagedLedger.
Expand Down
Expand Up @@ -18,7 +18,7 @@
*/
package org.apache.bookkeeper.mledger.impl;

import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
import org.apache.commons.lang3.tuple.Pair;
Expand Down Expand Up @@ -94,7 +94,7 @@ public interface EntryCache extends Comparable<EntryCache> {
* @param ctx
* the context object
*/
void asyncReadEntry(LedgerHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
ReadEntriesCallback callback, Object ctx);

/**
Expand All @@ -111,7 +111,7 @@ void asyncReadEntry(LedgerHandle lh, long firstEntry, long lastEntry, boolean is
* @param ctx
* the context object
*/
void asyncReadEntry(LedgerHandle lh, PositionImpl position, ReadEntryCallback callback, Object ctx);
void asyncReadEntry(ReadHandle lh, PositionImpl position, ReadEntryCallback callback, Object ctx);

/**
* Get the total size in bytes of all the entries stored in this cache.
Expand Down
Expand Up @@ -28,10 +28,11 @@
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.BKException;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
Expand Down Expand Up @@ -159,7 +160,7 @@ public void invalidateAllEntries(long ledgerId) {
}

@Override
public void asyncReadEntry(LedgerHandle lh, PositionImpl position, final ReadEntryCallback callback,
public void asyncReadEntry(ReadHandle lh, PositionImpl position, final ReadEntryCallback callback,
final Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}] Reading entry ledger {}: {}", ml.getName(), lh.getId(), position.getEntryId());
Expand All @@ -171,37 +172,38 @@ public void asyncReadEntry(LedgerHandle lh, PositionImpl position, final ReadEnt
manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength());
callback.readEntryComplete(cachedEntry, ctx);
} else {
lh.asyncReadEntries(position.getEntryId(), position.getEntryId(), (rc, ledgerHandle, sequence, obj) -> {
if (rc != BKException.Code.OK) {
ml.invalidateLedgerHandle(ledgerHandle, rc);
callback.readEntryFailed(createManagedLedgerException(rc), obj);
return;
}

if (sequence.hasMoreElements()) {
LedgerEntry ledgerEntry = sequence.nextElement();
EntryImpl returnEntry = EntryImpl.create(ledgerEntry);

// The EntryImpl is now the owner of the buffer, so we can release the original one
ledgerEntry.getEntryBuffer().release();

manager.mlFactoryMBean.recordCacheMiss(1, returnEntry.getLength());
ml.mbean.addReadEntriesSample(1, returnEntry.getLength());

ml.getExecutor().executeOrdered(ml.getName(), safeRun(() -> {
callback.readEntryComplete(returnEntry, obj);
}));
} else {
// got an empty sequence
callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), obj);
}
}, ctx);
lh.readAsync(position.getEntryId(), position.getEntryId()).whenCompleteAsync(
(ledgerEntries, exception) -> {
if (exception != null) {
ml.invalidateLedgerHandle(lh, exception);
callback.readEntryFailed(createManagedLedgerException(exception), ctx);
return;
}

try {
Iterator<LedgerEntry> iterator = ledgerEntries.iterator();
if (iterator.hasNext()) {
LedgerEntry ledgerEntry = iterator.next();
EntryImpl returnEntry = EntryImpl.create(ledgerEntry);

manager.mlFactoryMBean.recordCacheMiss(1, returnEntry.getLength());
ml.mbean.addReadEntriesSample(1, returnEntry.getLength());
callback.readEntryComplete(returnEntry, ctx);
} else {
// got an empty sequence
callback.readEntryFailed(new ManagedLedgerException("Could not read given position"),
ctx);
}
} finally {
ledgerEntries.close();
}
}, ml.getExecutor().chooseThread(ml.getName()));
}
}

@Override
@SuppressWarnings({ "unchecked", "rawtypes" })
public void asyncReadEntry(LedgerHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
final ReadEntriesCallback callback, Object ctx) {
final long ledgerId = lh.getId();
final int entriesToRead = (int) (lastEntry - firstEntry) + 1;
Expand Down Expand Up @@ -239,43 +241,43 @@ public void asyncReadEntry(LedgerHandle lh, long firstEntry, long lastEntry, boo
}

// Read all the entries from bookkeeper
lh.asyncReadEntries(firstEntry, lastEntry, (rc, lh1, sequence, cb) -> {

if (rc != BKException.Code.OK) {
if (rc == BKException.Code.TooManyRequestsException) {
callback.readEntriesFailed(createManagedLedgerException(rc), ctx);
} else {
ml.invalidateLedgerHandle(lh1, rc);
ManagedLedgerException mlException = createManagedLedgerException(rc);
callback.readEntriesFailed(mlException, ctx);
}
return;
}

checkNotNull(ml.getName());
checkNotNull(ml.getExecutor());
ml.getExecutor().executeOrdered(ml.getName(), safeRun(() -> {
// We got the entries, we need to transform them to a List<> type
long totalSize = 0;
final List<EntryImpl> entriesToReturn = Lists.newArrayListWithExpectedSize(entriesToRead);
while (sequence.hasMoreElements()) {
// Insert the entries at the end of the list (they will be unsorted for now)
LedgerEntry ledgerEntry = sequence.nextElement();
EntryImpl entry = EntryImpl.create(ledgerEntry);
ledgerEntry.getEntryBuffer().release();

entriesToReturn.add(entry);

totalSize += entry.getLength();

}

manager.mlFactoryMBean.recordCacheMiss(entriesToReturn.size(), totalSize);
ml.getMBean().addReadEntriesSample(entriesToReturn.size(), totalSize);

callback.readEntriesComplete((List) entriesToReturn, ctx);
}));
}, callback);
lh.readAsync(firstEntry, lastEntry).whenCompleteAsync(
(ledgerEntries, exception) -> {
if (exception != null) {
if (exception instanceof BKException
&& ((BKException)exception).getCode() == BKException.Code.TooManyRequestsException) {
callback.readEntriesFailed(createManagedLedgerException(exception), ctx);
} else {
ml.invalidateLedgerHandle(lh, exception);
ManagedLedgerException mlException = createManagedLedgerException(exception);
callback.readEntriesFailed(mlException, ctx);
}
return;
}

checkNotNull(ml.getName());
checkNotNull(ml.getExecutor());

try {
// We got the entries, we need to transform them to a List<> type
long totalSize = 0;
final List<EntryImpl> entriesToReturn
= Lists.newArrayListWithExpectedSize(entriesToRead);
for (LedgerEntry e : ledgerEntries) {
EntryImpl entry = EntryImpl.create(e);

entriesToReturn.add(entry);
totalSize += entry.getLength();
}

manager.mlFactoryMBean.recordCacheMiss(entriesToReturn.size(), totalSize);
ml.getMBean().addReadEntriesSample(entriesToReturn.size(), totalSize);

callback.readEntriesComplete((List) entriesToReturn, ctx);
} finally {
ledgerEntries.close();
}
}, ml.getExecutor().chooseThread(ml.getName()));
}
}

Expand Down
Expand Up @@ -33,8 +33,8 @@
import java.util.concurrent.atomic.AtomicLong;
import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback;
import org.apache.bookkeeper.mledger.Entry;
Expand Down Expand Up @@ -190,37 +190,35 @@ public Pair<Integer, Long> evictEntries(long sizeToFree) {
}

@Override
public void asyncReadEntry(LedgerHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader,
final ReadEntriesCallback callback, Object ctx) {
lh.asyncReadEntries(firstEntry, lastEntry, new ReadCallback() {
public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object bkctx) {
if (rc != BKException.Code.OK) {
callback.readEntriesFailed(createManagedLedgerException(rc), ctx);
return;
}

List<Entry> entries = Lists.newArrayList();
long totalSize = 0;
while (seq.hasMoreElements()) {
// Insert the entries at the end of the list (they will be unsorted for now)
LedgerEntry ledgerEntry = seq.nextElement();
EntryImpl entry = EntryImpl.create(ledgerEntry);
ledgerEntry.getEntryBuffer().release();

entries.add(entry);
totalSize += entry.getLength();
}

mlFactoryMBean.recordCacheMiss(entries.size(), totalSize);
ml.mbean.addReadEntriesSample(entries.size(), totalSize);

callback.readEntriesComplete(entries, null);
}
}, null);
lh.readAsync(firstEntry, lastEntry).whenComplete(
(ledgerEntries, exception) -> {
if (exception != null) {
callback.readEntriesFailed(createManagedLedgerException(exception), ctx);
return;
}
List<Entry> entries = Lists.newArrayList();
long totalSize = 0;
try {
for (LedgerEntry e : ledgerEntries) {
// Insert the entries at the end of the list (they will be unsorted for now)
EntryImpl entry = EntryImpl.create(e);
entries.add(entry);
totalSize += entry.getLength();
}
} finally {
ledgerEntries.close();
}
mlFactoryMBean.recordCacheMiss(entries.size(), totalSize);
ml.mbean.addReadEntriesSample(entries.size(), totalSize);

callback.readEntriesComplete(entries, null);
});
}

@Override
public void asyncReadEntry(LedgerHandle lh, PositionImpl position, AsyncCallbacks.ReadEntryCallback callback,
public void asyncReadEntry(ReadHandle lh, PositionImpl position, AsyncCallbacks.ReadEntryCallback callback,
Object ctx) {
}

Expand Down
Expand Up @@ -25,7 +25,7 @@
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
import io.netty.util.ReferenceCounted;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.mledger.Entry;

public final class EntryImpl extends AbstractReferenceCounted implements Entry, Comparable<EntryImpl>, ReferenceCounted {
Expand Down
Expand Up @@ -92,6 +92,7 @@ public class ManagedCursorImpl implements ManagedCursor {
protected final ManagedLedgerConfig config;
protected final ManagedLedgerImpl ledger;
private final String name;
private final BookKeeper.DigestType digestType;

protected volatile PositionImpl markDeletePosition;
protected volatile PositionImpl readPosition;
Expand Down Expand Up @@ -179,6 +180,7 @@ public interface VoidCallback {
this.config = config;
this.ledger = ledger;
this.name = cursorName;
this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType());
STATE_UPDATER.set(this, State.Uninitialized);
PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0);
PENDING_READ_OPS_UPDATER.set(this, 0);
Expand Down Expand Up @@ -253,7 +255,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac
// a new ledger and write the position into it
ledger.mbean.startCursorLedgerOpenOp();
long ledgerId = info.getCursorsLedgerId();
bookkeeper.asyncOpenLedger(ledgerId, config.getDigestType(), config.getPassword(), (rc, lh, ctx) -> {
bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), (rc, lh, ctx) -> {
if (log.isDebugEnabled()) {
log.debug("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc);
}
Expand Down Expand Up @@ -1924,7 +1926,7 @@ void createNewMetadataLedger(final VoidCallback callback) {
ledger.mbean.startCursorLedgerCreateOp();

bookkeeper.asyncCreateLedger(config.getMetadataEnsemblesize(), config.getMetadataWriteQuorumSize(),
config.getMetadataAckQuorumSize(), config.getDigestType(), config.getPassword(), (rc, lh, ctx) -> {
config.getMetadataAckQuorumSize(), digestType, config.getPassword(), (rc, lh, ctx) -> {
ledger.getExecutor().execute(safeRun(() -> {
ledger.mbean.endCursorLedgerCreateOp();
if (rc != BKException.Code.OK) {
Expand Down