-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[ManagedLedger] Compress managed ledger info #11490
Changes from 4 commits
66b87b3
d1d5ed6
108b2c2
816a65c
e565b97
52cf031
ffb8163
8bec01b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,15 +25,22 @@ | |
import java.util.Optional; | ||
import java.util.concurrent.CompletionException; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.CompositeByteBuf; | ||
import io.netty.buffer.Unpooled; | ||
import lombok.extern.slf4j.Slf4j; | ||
|
||
import org.apache.bookkeeper.common.util.OrderedExecutor; | ||
import org.apache.bookkeeper.mledger.ManagedLedgerException; | ||
import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; | ||
import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; | ||
import org.apache.bookkeeper.mledger.proto.MLDataFormats; | ||
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; | ||
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; | ||
import org.apache.bookkeeper.util.SafeRunnable; | ||
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; | ||
import org.apache.pulsar.common.api.proto.CompressionType; | ||
import org.apache.pulsar.common.compression.CompressionCodecProvider; | ||
import org.apache.pulsar.metadata.api.MetadataStore; | ||
import org.apache.pulsar.metadata.api.MetadataStoreException; | ||
import org.apache.pulsar.metadata.api.Stat; | ||
|
@@ -47,9 +54,31 @@ public class MetaStoreImpl implements MetaStore { | |
private final MetadataStore store; | ||
private final OrderedExecutor executor; | ||
|
||
public static final short MAGIC_MANAGED_LEDGER_INFO_METADATA = 0x0b9c; | ||
private final CompressionType compressionType; | ||
|
||
public MetaStoreImpl(MetadataStore store, OrderedExecutor executor) { | ||
this.store = store; | ||
this.executor = executor; | ||
this.compressionType = CompressionType.NONE; | ||
} | ||
|
||
public MetaStoreImpl(MetadataStore store, OrderedExecutor executor, String compressionType) { | ||
this.store = store; | ||
this.executor = executor; | ||
CompressionType finalCompressionType; | ||
if (compressionType != null) { | ||
try { | ||
finalCompressionType = CompressionType.valueOf(compressionType); | ||
} catch (Exception e) { | ||
log.warn("Failed to get compression type {}, disable managedLedgerInfo compression, error msg: {}.", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If the compression type is not valid we should just throw exception instead of trying to handle and disable compression. This could lead to situation in which someone thinks compression is enabled but it really is failing. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, I'll throw the exception. |
||
compressionType, e.getMessage()); | ||
finalCompressionType = CompressionType.NONE; | ||
} | ||
} else { | ||
finalCompressionType = CompressionType.NONE; | ||
} | ||
this.compressionType = finalCompressionType; | ||
} | ||
|
||
@Override | ||
|
@@ -62,7 +91,7 @@ public void getManagedLedgerInfo(String ledgerName, boolean createIfMissing, | |
if (optResult.isPresent()) { | ||
ManagedLedgerInfo info; | ||
try { | ||
info = ManagedLedgerInfo.parseFrom(optResult.get().getValue()); | ||
info = parseManagedLedgerInfo(optResult.get().getValue()); | ||
info = updateMLInfoTimestamp(info); | ||
callback.operationComplete(info, optResult.get().getStat()); | ||
} catch (InvalidProtocolBufferException e) { | ||
|
@@ -101,9 +130,8 @@ public void asyncUpdateLedgerIds(String ledgerName, ManagedLedgerInfo mlInfo, St | |
log.debug("[{}] Updating metadata version={} with content={}", ledgerName, stat, mlInfo); | ||
} | ||
|
||
byte[] serializedMlInfo = mlInfo.toByteArray(); // Binary format | ||
String path = PREFIX + ledgerName; | ||
store.put(path, serializedMlInfo, Optional.of(stat.getVersion())) | ||
store.put(path, compressLedgerInfo(mlInfo), Optional.of(stat.getVersion())) | ||
.thenAcceptAsync(newVersion -> callback.operationComplete(null, newVersion), executor.chooseThread(ledgerName)) | ||
.exceptionally(ex -> { | ||
executor.executeOrdered(ledgerName, SafeRunnable.safeRun(() -> callback.operationFailed(getException(ex)))); | ||
|
@@ -264,4 +292,84 @@ private static MetaStoreException getException(Throwable t) { | |
return new MetaStoreException(t); | ||
} | ||
} | ||
|
||
/** | ||
* Compress ManagedLedgerInfo data. | ||
* | ||
* compression data structure | ||
* [MAGIC_NUMBER](2) + [METADATA_SIZE](4) + [METADATA_PAYLOAD] + [MANAGED_LEDGER_INFO_PAYLOAD] | ||
*/ | ||
public byte[] compressLedgerInfo(ManagedLedgerInfo managedLedgerInfo) { | ||
if (compressionType == null || compressionType.equals(CompressionType.NONE)) { | ||
return managedLedgerInfo.toByteArray(); | ||
} | ||
ByteBuf metadataByteBuf = null; | ||
ByteBuf encodeByteBuf = null; | ||
try { | ||
MLDataFormats.ManagedLedgerInfoMetadata mlInfoMetadata = MLDataFormats.ManagedLedgerInfoMetadata | ||
.newBuilder() | ||
.setCompressionType(compressionType.name()) | ||
.setUnpressedSize(managedLedgerInfo.getSerializedSize()) | ||
.build(); | ||
metadataByteBuf = PulsarByteBufAllocator.DEFAULT.buffer( | ||
mlInfoMetadata.getSerializedSize() + 6, mlInfoMetadata.getSerializedSize() + 6); | ||
metadataByteBuf.writeShort(MAGIC_MANAGED_LEDGER_INFO_METADATA); | ||
metadataByteBuf.writeInt(mlInfoMetadata.getSerializedSize()); | ||
metadataByteBuf.writeBytes(mlInfoMetadata.toByteArray()); | ||
|
||
encodeByteBuf = CompressionCodecProvider.getCompressionCodec(compressionType) | ||
.encode(Unpooled.wrappedBuffer(managedLedgerInfo.toByteArray())); | ||
|
||
CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); | ||
compositeByteBuf.addComponent(true, metadataByteBuf); | ||
compositeByteBuf.addComponent(true, encodeByteBuf); | ||
byte[] dataBytes = new byte[compositeByteBuf.readableBytes()]; | ||
compositeByteBuf.readBytes(dataBytes); | ||
return dataBytes; | ||
} finally { | ||
if (metadataByteBuf != null) { | ||
metadataByteBuf.release(); | ||
} | ||
if (encodeByteBuf != null) { | ||
encodeByteBuf.release(); | ||
} | ||
} | ||
} | ||
|
||
public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws InvalidProtocolBufferException { | ||
ByteBuf byteBuf = Unpooled.wrappedBuffer(data); | ||
if (byteBuf.readableBytes() > 0 && byteBuf.readShort() == MAGIC_MANAGED_LEDGER_INFO_METADATA) { | ||
ByteBuf decodeByteBuf = null; | ||
try { | ||
int metadataSize = byteBuf.readInt(); | ||
byte[] metadataBytes = new byte[metadataSize]; | ||
byteBuf.readBytes(metadataBytes); | ||
MLDataFormats.ManagedLedgerInfoMetadata metadata = | ||
MLDataFormats.ManagedLedgerInfoMetadata.parseFrom(metadataBytes); | ||
|
||
long unpressedSize = metadata.getUnpressedSize(); | ||
decodeByteBuf = CompressionCodecProvider.getCompressionCodec( | ||
CompressionType.valueOf(metadata.getCompressionType())).decode(byteBuf, (int) unpressedSize); | ||
byte[] decodeBytes; | ||
if (decodeByteBuf.hasArray() && !metadata.getCompressionType().equals(CompressionType.ZLIB.name())) { | ||
decodeBytes = decodeByteBuf.array(); | ||
} else { | ||
decodeBytes = new byte[decodeByteBuf.readableBytes() - decodeByteBuf.readerIndex()]; | ||
decodeByteBuf.readBytes(decodeBytes); | ||
} | ||
return ManagedLedgerInfo.parseFrom(decodeBytes); | ||
} catch (Exception e) { | ||
log.error("Failed to parse managedLedgerInfo metadata, " | ||
+ "fall back to parse managedLedgerInfo directly.", e); | ||
return ManagedLedgerInfo.parseFrom(data); | ||
} finally { | ||
if (decodeByteBuf != null) { | ||
decodeByteBuf.release(); | ||
} | ||
} | ||
} else { | ||
return ManagedLedgerInfo.parseFrom(data); | ||
} | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -124,3 +124,8 @@ message ManagedCursorInfo { | |||||
// Store which index in the batch message has been deleted | ||||||
repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 7; | ||||||
} | ||||||
|
||||||
message ManagedLedgerInfoMetadata { | ||||||
required string compressionType = 1; | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use compression enum type instead of String? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we could just copy the same enum type here, or even declare it as There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looks we'd better to copy the enum type to the MLDataFormats.proto because the managed ledger module can be a independent component. |
||||||
required int32 unpressedSize = 2; | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks. |
||||||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,113 @@ | ||
/** | ||
* 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.mledger.impl; | ||
|
||
import com.google.protobuf.InvalidProtocolBufferException; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.bookkeeper.mledger.offload.OffloadUtils; | ||
import org.apache.bookkeeper.mledger.proto.MLDataFormats; | ||
import org.apache.commons.lang3.RandomUtils; | ||
import org.apache.pulsar.common.api.proto.CompressionType; | ||
import org.testng.Assert; | ||
import org.testng.annotations.DataProvider; | ||
import org.testng.annotations.Test; | ||
|
||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.UUID; | ||
|
||
/** | ||
* ManagedLedgerInfo metadata test. | ||
*/ | ||
@Slf4j | ||
public class ManagedLedgerInfoMetadataTest { | ||
|
||
@DataProvider(name = "compressionTypeProvider") | ||
private Object[][] compressionTypeProvider() { | ||
return new Object[][] { | ||
{null}, | ||
{"INVALID_TYPE"}, | ||
{CompressionType.NONE.name()}, | ||
{CompressionType.LZ4.name()}, | ||
{CompressionType.ZLIB.name()}, | ||
{CompressionType.ZSTD.name()}, | ||
{CompressionType.SNAPPY.name()} | ||
}; | ||
} | ||
|
||
@Test(dataProvider = "compressionTypeProvider") | ||
public void testEncodeAndDecode(String compressionType) throws IOException { | ||
long ledgerId = 10000; | ||
List<MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgerInfoList = new ArrayList<>(); | ||
for (int i = 0; i < 100; i++) { | ||
MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder builder = MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder(); | ||
builder.setLedgerId(ledgerId); | ||
builder.setEntries(RandomUtils.nextInt()); | ||
builder.setSize(RandomUtils.nextLong()); | ||
builder.setTimestamp(System.currentTimeMillis()); | ||
|
||
UUID uuid = UUID.randomUUID(); | ||
builder.getOffloadContextBuilder() | ||
.setUidMsb(uuid.getMostSignificantBits()) | ||
.setUidLsb(uuid.getLeastSignificantBits()); | ||
Map<String, String> offloadDriverMetadata = new HashMap<>(); | ||
offloadDriverMetadata.put("bucket", "test-bucket"); | ||
offloadDriverMetadata.put("managedLedgerOffloadDriver", "pulsar-offload-dev"); | ||
offloadDriverMetadata.put("serviceEndpoint", "https://s3.eu-west-1.amazonaws.com"); | ||
offloadDriverMetadata.put("region", "eu-west-1"); | ||
OffloadUtils.setOffloadDriverMetadata( | ||
builder, | ||
"aws-s3", | ||
offloadDriverMetadata | ||
); | ||
|
||
MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = builder.build(); | ||
ledgerInfoList.add(ledgerInfo); | ||
ledgerId ++; | ||
} | ||
|
||
MLDataFormats.ManagedLedgerInfo managedLedgerInfo = MLDataFormats.ManagedLedgerInfo.newBuilder() | ||
.addAllLedgerInfo(ledgerInfoList) | ||
.build(); | ||
MetaStoreImpl metaStore = new MetaStoreImpl(null, null, compressionType); | ||
byte[] compressionBytes = metaStore.compressLedgerInfo(managedLedgerInfo); | ||
log.info("[{}] Uncompressed data size: {}, compressed data size: {}", | ||
compressionType, managedLedgerInfo.getSerializedSize(), compressionBytes.length); | ||
if (compressionType == null || compressionType.equals(CompressionType.NONE.name()) | ||
|| compressionType.equals("INVALID_TYPE")) { | ||
Assert.assertEquals(compressionBytes.length, managedLedgerInfo.getSerializedSize()); | ||
} | ||
|
||
// parse compression data and unCompression data, check their results. | ||
MLDataFormats.ManagedLedgerInfo info1 = metaStore.parseManagedLedgerInfo(compressionBytes); | ||
MLDataFormats.ManagedLedgerInfo info2 = metaStore.parseManagedLedgerInfo(managedLedgerInfo.toByteArray()); | ||
Assert.assertEquals(info1, info2); | ||
} | ||
|
||
@Test | ||
public void testParseEmptyData() throws InvalidProtocolBufferException { | ||
MetaStoreImpl metaStore = new MetaStoreImpl(null, null); | ||
MLDataFormats.ManagedLedgerInfo managedLedgerInfo = metaStore.parseManagedLedgerInfo(new byte[0]); | ||
Assert.assertEquals(managedLedgerInfo.toString(), ""); | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
private?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should try to use a magic number with the most significant bit set to 1 because that should be guaranteed to be a non-valid protobuf sequence
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, I'll change this.