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

[fix][offload] Fix Offload readHandle cannot close multi times. #22162

Merged
merged 6 commits into from
Feb 29, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -347,4 +347,21 @@ public static void safeRunAsync(Runnable runnable,
return null;
});
}

public static <T> @Nonnull CompletableFuture<T> apply(@Nonnull CompletableFuture<T> s,
dao-jun marked this conversation as resolved.
Show resolved Hide resolved
@Nonnull CompletableFuture<T> d) {
if (s == d) {
return d;
}

s.whenComplete((v, t) -> {
if (t == null) {
d.complete(v);
} else {
d.completeExceptionally(t);
}
});

return d;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
import org.apache.bookkeeper.client.api.LedgerEntries;
Expand All @@ -36,11 +37,13 @@
import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
import org.apache.bookkeeper.mledger.LedgerOffloaderStats;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.MapFile;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -53,6 +56,12 @@ public class FileStoreBackedReadHandleImpl implements ReadHandle {
private final LedgerOffloaderStats offloaderStats;
private final String managedLedgerName;
private final String topicName;
enum State {
Opened,
Closed
}
private volatile State state;
private final AtomicReference<CompletableFuture<Void>> closeFuture = new AtomicReference<>();

private FileStoreBackedReadHandleImpl(ExecutorService executor, MapFile.Reader reader, long ledgerId,
LedgerOffloaderStats offloaderStats,
Expand All @@ -72,6 +81,7 @@ private FileStoreBackedReadHandleImpl(ExecutorService executor, MapFile.Reader r
offloaderStats.recordReadOffloadIndexLatency(topicName,
System.nanoTime() - startReadIndexTime, TimeUnit.NANOSECONDS);
this.ledgerMetadata = parseLedgerMetadata(ledgerId, value.copyBytes());
state = State.Opened;
} catch (IOException e) {
log.error("Fail to read LedgerMetadata for ledgerId {}",
ledgerId);
Expand All @@ -93,14 +103,20 @@ public LedgerMetadata getLedgerMetadata() {
@Override
public CompletableFuture<Void> closeAsync() {
CompletableFuture<Void> promise = new CompletableFuture<>();

if (!closeFuture.compareAndSet(null, promise)) {
return FutureUtil.apply(closeFuture.get(), promise);
}
lhotari marked this conversation as resolved.
Show resolved Hide resolved

executor.execute(() -> {
try {
reader.close();
promise.complete(null);
} catch (IOException t) {
promise.completeExceptionally(t);
}
});
try {
reader.close();
state = State.Closed;
promise.complete(null);
} catch (IOException t) {
promise.completeExceptionally(t);
}
});
return promise;
}

Expand All @@ -111,6 +127,12 @@ public CompletableFuture<LedgerEntries> readAsync(long firstEntry, long lastEntr
}
CompletableFuture<LedgerEntries> promise = new CompletableFuture<>();
executor.execute(() -> {
if (state == State.Closed) {
log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}",
ledgerId, firstEntry, lastEntry);
promise.completeExceptionally(new ManagedLedgerException.OffloadReadHandleClosedException());
return;
}
if (firstEntry > lastEntry
|| firstEntry < 0
|| lastEntry > getLastAddConfirmed()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.bookkeeper.mledger.offload.jcloud.impl;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import io.netty.buffer.ByteBuf;
Expand All @@ -30,6 +31,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
import org.apache.bookkeeper.client.api.LedgerEntries;
Expand All @@ -46,6 +48,7 @@
import org.apache.bookkeeper.mledger.offload.jcloud.impl.DataBlockUtils.VersionCheck;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.domain.Blob;
import org.slf4j.Logger;
Expand All @@ -66,13 +69,14 @@ public class BlobStoreBackedReadHandleImpl implements ReadHandle {
.newBuilder()
.expireAfterAccess(CACHE_TTL_SECONDS, TimeUnit.SECONDS)
.build();
private final AtomicReference<CompletableFuture<Void>> closeFuture = new AtomicReference<>();

enum State {
Opened,
Closed
}

private State state = null;
private volatile State state = null;

private BlobStoreBackedReadHandleImpl(long ledgerId, OffloadIndexBlock index,
BackedInputStream inputStream, ExecutorService executor) {
Expand All @@ -97,17 +101,22 @@ public LedgerMetadata getLedgerMetadata() {
@Override
public CompletableFuture<Void> closeAsync() {
CompletableFuture<Void> promise = new CompletableFuture<>();

if (!closeFuture.compareAndSet(null, promise)) {
return FutureUtil.apply(closeFuture.get(), promise);
}

executor.execute(() -> {
try {
index.close();
inputStream.close();
entryOffsets.invalidateAll();
state = State.Closed;
promise.complete(null);
} catch (IOException t) {
promise.completeExceptionally(t);
}
});
try {
index.close();
inputStream.close();
entryOffsets.invalidateAll();
state = State.Closed;
promise.complete(null);
} catch (IOException t) {
promise.completeExceptionally(t);
}
});
return promise;
}

Expand Down Expand Up @@ -298,6 +307,7 @@ public static ReadHandle open(ScheduledExecutorService executor,
}

// for testing
@VisibleForTesting
State getState() {
return this.state;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import lombok.val;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
Expand All @@ -47,6 +48,7 @@
import org.apache.bookkeeper.mledger.offload.jcloud.impl.DataBlockUtils.VersionCheck;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.domain.Blob;
import org.slf4j.Logger;
Expand All @@ -60,7 +62,8 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle {
private final List<BackedInputStream> inputStreams;
private final List<DataInputStream> dataStreams;
private final ExecutorService executor;
private State state = null;
private volatile State state = null;
private final AtomicReference<CompletableFuture<Void>> closeFuture = new AtomicReference<>();

enum State {
Opened,
Expand Down Expand Up @@ -124,6 +127,11 @@ public LedgerMetadata getLedgerMetadata() {
@Override
public CompletableFuture<Void> closeAsync() {
CompletableFuture<Void> promise = new CompletableFuture<>();

if (!closeFuture.compareAndSet(null, promise)) {
return FutureUtil.apply(closeFuture.get(), promise);
}

executor.execute(() -> {
try {
for (OffloadIndexBlockV2 indexBlock : indices) {
Expand Down
Loading