Skip to content

Commit f0b1471

Browse files
ivankellysijie
authored andcommitted
ReadHandle implementation backed by S3 (#1790)
Implementation of bookkeeper ReadHandle, which reads from an S3 object. Master Issue: #1511
1 parent b47c059 commit f0b1471

File tree

7 files changed

+421
-127
lines changed

7 files changed

+421
-127
lines changed

conf/broker.conf

Lines changed: 3 additions & 0 deletions
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -495,6 +495,9 @@ s3ManagedLedgerOffloadServiceEndpoint=
495
# For Amazon S3 ledger offload, Max block size in bytes.
495
# For Amazon S3 ledger offload, Max block size in bytes.
496
s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864
496
s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864
497

497

498+
# For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default)
499+
s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576
500+
498
### --- Deprecated config variables --- ###
501
### --- Deprecated config variables --- ###
499

502

500
# Deprecated. Use configurationStoreServers
503
# Deprecated. Use configurationStoreServers

pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java

Lines changed: 12 additions & 0 deletions
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -487,6 +487,10 @@ public class ServiceConfiguration implements PulsarConfiguration {
487
// For Amazon S3 ledger offload, Max block size in bytes.
487
// For Amazon S3 ledger offload, Max block size in bytes.
488
private int s3ManagedLedgerOffloadMaxBlockSizeInBytes = 64 * 1024 * 1024;
488
private int s3ManagedLedgerOffloadMaxBlockSizeInBytes = 64 * 1024 * 1024;
489

489

490+
// For Amazon S3 ledger offload, Read buffer size in bytes.
491+
@FieldContext(minValue = 1024)
492+
private int s3ManagedLedgerOffloadReadBufferSizeInBytes = 1024 * 1024; // 1MB
493+
490
public String getZookeeperServers() {
494
public String getZookeeperServers() {
491
return zookeeperServers;
495
return zookeeperServers;
492
}
496
}
@@ -1694,4 +1698,12 @@ public int getS3ManagedLedgerOffloadMaxBlockSizeInBytes() {
1694
return this.s3ManagedLedgerOffloadMaxBlockSizeInBytes;
1698
return this.s3ManagedLedgerOffloadMaxBlockSizeInBytes;
1695
}
1699
}
1696

1700

1701+
public void setS3ManagedLedgerOffloadReadBufferSizeInBytes(int readBufferSizeInBytes) {
1702+
this.s3ManagedLedgerOffloadReadBufferSizeInBytes = readBufferSizeInBytes;
1703+
}
1704+
1705+
public int getS3ManagedLedgerOffloadReadBufferSizeInBytes() {
1706+
return this.s3ManagedLedgerOffloadReadBufferSizeInBytes;
1707+
}
1708+
1697
}
1709
}

pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/OffloadIndexEntry.java

Lines changed: 6 additions & 1 deletion
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -43,8 +43,13 @@ public interface OffloadIndexEntry {
43
int getPartId();
43
int getPartId();
44

44

45
/**
45
/**
46-
* Get the offset of this message entry in code storage.
46+
* Get the offset of this block within the object.
47
*/
47
*/
48
long getOffset();
48
long getOffset();
49+
50+
/**
51+
* Get the offset of the block's data within the object.
52+
*/
53+
long getDataOffset();
49
}
54
}
50

55

pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/S3ManagedLedgerOffloader.java

Lines changed: 26 additions & 11 deletions
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -45,6 +45,7 @@
45
import org.apache.pulsar.broker.PulsarServerException;
45
import org.apache.pulsar.broker.PulsarServerException;
46
import org.apache.pulsar.broker.ServiceConfiguration;
46
import org.apache.pulsar.broker.ServiceConfiguration;
47
import org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
47
import org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
48+
import org.apache.pulsar.broker.s3offload.impl.S3BackedReadHandleImpl;
48
import org.slf4j.Logger;
49
import org.slf4j.Logger;
49
import org.slf4j.LoggerFactory;
50
import org.slf4j.LoggerFactory;
50

51

@@ -57,6 +58,7 @@ public class S3ManagedLedgerOffloader implements LedgerOffloader {
57
private final String bucket;
58
private final String bucket;
58
// max block size for each data block.
59
// max block size for each data block.
59
private int maxBlockSize;
60
private int maxBlockSize;
61+
private final int readBufferSize;
60

62

61
public static S3ManagedLedgerOffloader create(ServiceConfiguration conf,
63
public static S3ManagedLedgerOffloader create(ServiceConfiguration conf,
62
ScheduledExecutorService scheduler)
64
ScheduledExecutorService scheduler)
@@ -65,6 +67,7 @@ public static S3ManagedLedgerOffloader create(ServiceConfiguration conf,
65
String bucket = conf.getS3ManagedLedgerOffloadBucket();
67
String bucket = conf.getS3ManagedLedgerOffloadBucket();
66
String endpoint = conf.getS3ManagedLedgerOffloadServiceEndpoint();
68
String endpoint = conf.getS3ManagedLedgerOffloadServiceEndpoint();
67
int maxBlockSize = conf.getS3ManagedLedgerOffloadMaxBlockSizeInBytes();
69
int maxBlockSize = conf.getS3ManagedLedgerOffloadMaxBlockSizeInBytes();
70+
int readBufferSize = conf.getS3ManagedLedgerOffloadReadBufferSizeInBytes();
68

71

69
if (Strings.isNullOrEmpty(region)) {
72
if (Strings.isNullOrEmpty(region)) {
70
throw new PulsarServerException("s3ManagedLedgerOffloadRegion cannot be empty is s3 offload enabled");
73
throw new PulsarServerException("s3ManagedLedgerOffloadRegion cannot be empty is s3 offload enabled");
@@ -80,22 +83,24 @@ public static S3ManagedLedgerOffloader create(ServiceConfiguration conf,
80
} else {
83
} else {
81
builder.setRegion(region);
84
builder.setRegion(region);
82
}
85
}
83-
return new S3ManagedLedgerOffloader(builder.build(), bucket, scheduler, maxBlockSize);
86+
return new S3ManagedLedgerOffloader(builder.build(), bucket, scheduler, maxBlockSize, readBufferSize);
84
}
87
}
85

88

86-
S3ManagedLedgerOffloader(AmazonS3 s3client, String bucket, ScheduledExecutorService scheduler, int maxBlockSize) {
89+
S3ManagedLedgerOffloader(AmazonS3 s3client, String bucket, ScheduledExecutorService scheduler,
90+
int maxBlockSize, int readBufferSize) {
87
this.s3client = s3client;
91
this.s3client = s3client;
88
this.bucket = bucket;
92
this.bucket = bucket;
89
this.scheduler = scheduler;
93
this.scheduler = scheduler;
90
this.maxBlockSize = maxBlockSize;
94
this.maxBlockSize = maxBlockSize;
95+
this.readBufferSize = readBufferSize;
91
}
96
}
92

97

93-
static String dataBlockOffloadKey(ReadHandle readHandle, UUID uuid) {
98+
static String dataBlockOffloadKey(long ledgerId, UUID uuid) {
94-
return String.format("ledger-%d-%s", readHandle.getId(), uuid.toString());
99+
return String.format("ledger-%d-%s", ledgerId, uuid.toString());
95
}
100
}
96

101

97-
static String indexBlockOffloadKey(ReadHandle readHandle, UUID uuid) {
102+
static String indexBlockOffloadKey(long ledgerId, UUID uuid) {
98-
return String.format("ledger-%d-%s-index", readHandle.getId(), uuid.toString());
103+
return String.format("ledger-%d-%s-index", ledgerId, uuid.toString());
99
}
104
}
100

105

101
// upload DataBlock to s3 using MultiPartUpload, and indexBlock in a new Block,
106
// upload DataBlock to s3 using MultiPartUpload, and indexBlock in a new Block,
@@ -107,8 +112,8 @@ public CompletableFuture<Void> offload(ReadHandle readHandle,
107
scheduler.submit(() -> {
112
scheduler.submit(() -> {
108
OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create()
113
OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create()
109
.withMetadata(readHandle.getLedgerMetadata());
114
.withMetadata(readHandle.getLedgerMetadata());
110-
String dataBlockKey = dataBlockOffloadKey(readHandle, uuid);
115+
String dataBlockKey = dataBlockOffloadKey(readHandle.getId(), uuid);
111-
String indexBlockKey = indexBlockOffloadKey(readHandle, uuid);
116+
String indexBlockKey = indexBlockOffloadKey(readHandle.getId(), uuid);
112
InitiateMultipartUploadRequest dataBlockReq = new InitiateMultipartUploadRequest(bucket, dataBlockKey);
117
InitiateMultipartUploadRequest dataBlockReq = new InitiateMultipartUploadRequest(bucket, dataBlockKey);
113
InitiateMultipartUploadResult dataBlockRes = null;
118
InitiateMultipartUploadResult dataBlockRes = null;
114

119

@@ -174,12 +179,12 @@ public CompletableFuture<Void> offload(ReadHandle readHandle,
174
metadata.setContentLength(indexStream.available());
179
metadata.setContentLength(indexStream.available());
175
s3client.putObject(new PutObjectRequest(
180
s3client.putObject(new PutObjectRequest(
176
bucket,
181
bucket,
177-
indexBlockOffloadKey(readHandle, uuid),
182+
indexBlockKey,
178
indexStream,
183
indexStream,
179
metadata));
184
metadata));
180
promise.complete(null);
185
promise.complete(null);
181
} catch (Throwable t) {
186
} catch (Throwable t) {
182-
s3client.deleteObject(bucket, dataBlockOffloadKey(readHandle, uuid));
187+
s3client.deleteObject(bucket, dataBlockKey);
183
promise.completeExceptionally(t);
188
promise.completeExceptionally(t);
184
return;
189
return;
185
}
190
}
@@ -190,7 +195,17 @@ public CompletableFuture<Void> offload(ReadHandle readHandle,
190
@Override
195
@Override
191
public CompletableFuture<ReadHandle> readOffloaded(long ledgerId, UUID uid) {
196
public CompletableFuture<ReadHandle> readOffloaded(long ledgerId, UUID uid) {
192
CompletableFuture<ReadHandle> promise = new CompletableFuture<>();
197
CompletableFuture<ReadHandle> promise = new CompletableFuture<>();
193-
promise.completeExceptionally(new UnsupportedOperationException());
198+
String key = dataBlockOffloadKey(ledgerId, uid);
199+
String indexKey = indexBlockOffloadKey(ledgerId, uid);
200+
scheduler.submit(() -> {
201+
try {
202+
promise.complete(S3BackedReadHandleImpl.open(scheduler, s3client,
203+
bucket, key, indexKey,
204+
ledgerId, readBufferSize));
205+
} catch (Throwable t) {
206+
promise.completeExceptionally(t);
207+
}
208+
});
194
return promise;
209
return promise;
195
}
210
}
196

211

pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/OffloadIndexEntryImpl.java

Lines changed: 4 additions & 0 deletions
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -48,6 +48,10 @@ public int getPartId() {
48
public long getOffset() {
48
public long getOffset() {
49
return offset;
49
return offset;
50
}
50
}
51+
@Override
52+
public long getDataOffset() {
53+
return offset + DataBlockHeaderImpl.getDataStartOffset();
54+
}
51

55

52
public OffloadIndexEntryImpl(long entryId, int partId, long offset) {
56
public OffloadIndexEntryImpl(long entryId, int partId, long offset) {
53
this.entryId = entryId;
57
this.entryId = entryId;
Lines changed: 207 additions & 0 deletions
Original file line numberOriginal file lineDiff line numberDiff line change
@@ -0,0 +1,207 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.pulsar.broker.s3offload.impl;
20+
21+
import com.amazonaws.AmazonClientException;
22+
import com.amazonaws.services.s3.AmazonS3;
23+
import com.amazonaws.services.s3.model.GetObjectRequest;
24+
import com.amazonaws.services.s3.model.ObjectMetadata;
25+
import com.amazonaws.services.s3.model.S3Object;
26+
27+
import io.netty.buffer.ByteBuf;
28+
import io.netty.buffer.PooledByteBufAllocator;
29+
30+
import java.io.DataInputStream;
31+
import java.io.IOException;
32+
import java.util.ArrayList;
33+
import java.util.List;
34+
import java.util.concurrent.CompletableFuture;
35+
import java.util.concurrent.ExecutorService;
36+
import java.util.concurrent.ScheduledExecutorService;
37+
38+
import org.apache.bookkeeper.client.BKException;
39+
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
40+
import org.apache.bookkeeper.client.api.LedgerEntries;
41+
import org.apache.bookkeeper.client.api.LedgerEntry;
42+
import org.apache.bookkeeper.client.api.LedgerMetadata;
43+
import org.apache.bookkeeper.client.api.ReadHandle;
44+
import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
45+
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
46+
47+
import org.apache.pulsar.broker.s3offload.OffloadIndexBlock;
48+
import org.apache.pulsar.broker.s3offload.OffloadIndexBlockBuilder;
49+
import org.apache.pulsar.broker.s3offload.OffloadIndexEntry;
50+
import org.apache.pulsar.broker.s3offload.S3BackedInputStream;
51+
52+
import org.slf4j.Logger;
53+
import org.slf4j.LoggerFactory;
54+
55+
public class S3BackedReadHandleImpl implements ReadHandle {
56+
private static final Logger log = LoggerFactory.getLogger(S3BackedReadHandleImpl.class);
57+
58+
private final long ledgerId;
59+
private final OffloadIndexBlock index;
60+
private final S3BackedInputStream inputStream;
61+
private final DataInputStream dataStream;
62+
private final ExecutorService executor;
63+
64+
private S3BackedReadHandleImpl(long ledgerId, OffloadIndexBlock index,
65+
S3BackedInputStream inputStream,
66+
ExecutorService executor) {
67+
this.ledgerId = ledgerId;
68+
this.index = index;
69+
this.inputStream = inputStream;
70+
this.dataStream = new DataInputStream(inputStream);
71+
this.executor = executor;
72+
}
73+
74+
@Override
75+
public long getId() {
76+
return ledgerId;
77+
}
78+
79+
@Override
80+
public LedgerMetadata getLedgerMetadata() {
81+
return index.getLedgerMetadata();
82+
}
83+
84+
@Override
85+
public CompletableFuture<Void> closeAsync() {
86+
CompletableFuture<Void> promise = new CompletableFuture<>();
87+
executor.submit(() -> {
88+
try {
89+
index.close();
90+
inputStream.close();
91+
promise.complete(null);
92+
} catch (IOException t) {
93+
promise.completeExceptionally(t);
94+
}
95+
});
96+
return promise;
97+
}
98+
99+
@Override
100+
public CompletableFuture<LedgerEntries> readAsync(long firstEntry, long lastEntry) {
101+
log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry);
102+
CompletableFuture<LedgerEntries> promise = new CompletableFuture<>();
103+
executor.submit(() -> {
104+
if (firstEntry > lastEntry
105+
|| firstEntry < 0
106+
|| lastEntry > getLastAddConfirmed()) {
107+
promise.completeExceptionally(new BKException.BKIncorrectParameterException());
108+
return;
109+
}
110+
long entriesToRead = (lastEntry - firstEntry) + 1;
111+
List<LedgerEntry> entries = new ArrayList<LedgerEntry>();
112+
long nextExpectedId = firstEntry;
113+
try {
114+
OffloadIndexEntry entry = index.getIndexEntryForEntry(firstEntry);
115+
inputStream.seek(entry.getDataOffset());
116+
117+
while (entriesToRead > 0) {
118+
int length = dataStream.readInt();
119+
if (length < 0) { // hit padding or new block
120+
inputStream.seekForward(index.getIndexEntryForEntry(nextExpectedId).getDataOffset());
121+
length = dataStream.readInt();
122+
}
123+
long entryId = dataStream.readLong();
124+
125+
if (entryId == nextExpectedId) {
126+
ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(length, length);
127+
entries.add(LedgerEntryImpl.create(ledgerId, entryId, length, buf));
128+
int toWrite = length;
129+
while (toWrite > 0) {
130+
toWrite -= buf.writeBytes(dataStream, toWrite);
131+
}
132+
entriesToRead--;
133+
nextExpectedId++;
134+
} else if (entryId > lastEntry) {
135+
log.info("Expected to read {}, but read {}, which is greater than last entry {}",
136+
nextExpectedId, entryId, lastEntry);
137+
throw new BKException.BKUnexpectedConditionException();
138+
} else {
139+
inputStream.skip(length);
140+
}
141+
}
142+
143+
promise.complete(LedgerEntriesImpl.create(entries));
144+
} catch (Throwable t) {
145+
promise.completeExceptionally(t);
146+
entries.forEach(LedgerEntry::close);
147+
}
148+
});
149+
return promise;
150+
}
151+
152+
@Override
153+
public CompletableFuture<LedgerEntries> readUnconfirmedAsync(long firstEntry, long lastEntry) {
154+
return readAsync(firstEntry, lastEntry);
155+
}
156+
157+
@Override
158+
public CompletableFuture<Long> readLastAddConfirmedAsync() {
159+
return CompletableFuture.completedFuture(getLastAddConfirmed());
160+
}
161+
162+
@Override
163+
public CompletableFuture<Long> tryReadLastAddConfirmedAsync() {
164+
return CompletableFuture.completedFuture(getLastAddConfirmed());
165+
}
166+
167+
@Override
168+
public long getLastAddConfirmed() {
169+
return getLedgerMetadata().getLastEntryId();
170+
}
171+
172+
@Override
173+
public long getLength() {
174+
return getLedgerMetadata().getLength();
175+
}
176+
177+
@Override
178+
public boolean isClosed() {
179+
return getLedgerMetadata().isClosed();
180+
}
181+
182+
@Override
183+
public CompletableFuture<LastConfirmedAndEntry> readLastAddConfirmedAndEntryAsync(long entryId,
184+
long timeOutInMillis,
185+
boolean parallel) {
186+
CompletableFuture<LastConfirmedAndEntry> promise = new CompletableFuture<>();
187+
promise.completeExceptionally(new UnsupportedOperationException());
188+
return promise;
189+
}
190+
191+
public static ReadHandle open(ScheduledExecutorService executor,
192+
AmazonS3 s3client, String bucket, String key, String indexKey,
193+
long ledgerId, int readBufferSize)
194+
throws AmazonClientException, IOException {
195+
GetObjectRequest req = new GetObjectRequest(bucket, indexKey);
196+
try (S3Object obj = s3client.getObject(req)) {
197+
OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create();
198+
OffloadIndexBlock index = indexBuilder.fromStream(obj.getObjectContent());
199+
200+
ObjectMetadata dataMetadata = s3client.getObjectMetadata(bucket, key); // FIXME: this should be part of index
201+
S3BackedInputStream inputStream = new S3BackedInputStreamImpl(s3client, bucket, key,
202+
dataMetadata.getContentLength(),
203+
readBufferSize);
204+
return new S3BackedReadHandleImpl(ledgerId, index, inputStream, executor);
205+
}
206+
}
207+
}

0 commit comments

Comments
 (0)