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

[ManagedLedger] Compress managed ledger info #11490

Merged
merged 8 commits into from Aug 4, 2021
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -25,6 +25,9 @@
import java.util.Optional;
import java.util.concurrent.CompletionException;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
import io.netty.util.concurrent.FastThreadLocal;
import lombok.extern.slf4j.Slf4j;

import org.apache.bookkeeper.common.util.OrderedExecutor;
Expand All @@ -34,6 +37,10 @@
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.api.proto.ManagedLedgerInfoMetadata;
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;
Expand All @@ -47,6 +54,15 @@ public class MetaStoreImpl implements MetaStore {
private final MetadataStore store;
private final OrderedExecutor executor;

public static final short magicManagedLedgerInfoMetadata = 0x0b9c;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
public static final short magicManagedLedgerInfoMetadata = 0x0b9c;
public static final short MAGIC_MANAGED_LEDGER_INFO_METADATA = 0x0b9c;

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll fix this.

private final FastThreadLocal<ManagedLedgerInfoMetadata> ML_INFO_METADATA =
new FastThreadLocal<ManagedLedgerInfoMetadata>() {
@Override
protected ManagedLedgerInfoMetadata initialValue() throws Exception {
return new ManagedLedgerInfoMetadata();
}
};

public MetaStoreImpl(MetadataStore store, OrderedExecutor executor) {
this.store = store;
this.executor = executor;
Expand All @@ -62,7 +78,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) {
Expand Down Expand Up @@ -101,9 +117,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, CompressionType.ZSTD), Optional.of(stat.getVersion()))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The compression type should be configurable

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good idea.

.thenAcceptAsync(newVersion -> callback.operationComplete(null, newVersion), executor.chooseThread(ledgerName))
.exceptionally(ex -> {
executor.executeOrdered(ledgerName, SafeRunnable.safeRun(() -> callback.operationFailed(getException(ex))));
Expand Down Expand Up @@ -264,4 +279,59 @@ private static MetaStoreException getException(Throwable t) {
return new MetaStoreException(t);
}
}

/**
* [MAGIC_NUMBER](2) +
* [METADATA_SIZE](4) +
* [METADATA](METADATA_SIZE) +
* [MANAGED_LEDGER_INFO_PAYLOAD](MANAGED_LEDGER_INFO_PAYLOAD_SIZE)
*/
public byte[] compressLedgerInfo(ManagedLedgerInfo managedLedgerInfo, CompressionType compressionType) {
byte[] originalBytes = managedLedgerInfo.toByteArray();
ManagedLedgerInfoMetadata mlInfoMetadata = ML_INFO_METADATA.get();
mlInfoMetadata.setCompressionType(compressionType);
mlInfoMetadata.setUnpressedSize(originalBytes.length);
ByteBuf metadataByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(
mlInfoMetadata.getSerializedSize() + 6, mlInfoMetadata.getSerializedSize() + 6);
metadataByteBuf.writeShort(magicManagedLedgerInfoMetadata);
metadataByteBuf.writeInt(mlInfoMetadata.getSerializedSize());
mlInfoMetadata.writeTo(metadataByteBuf);

ByteBuf originalByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(originalBytes.length, originalBytes.length);
originalByteBuf.writeBytes(originalBytes);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems we can use wrappedBuffer to reduce one copy

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good idea.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, I'll fix this.

ByteBuf encodeByteBuf = CompressionCodecProvider.getCompressionCodec(compressionType).encode(originalByteBuf);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

/**
     * Compress a buffer.
     *
     * @param raw
     *            a buffer with the uncompressed content. The reader/writer indexes will not be modified
     * @return a new buffer with the compressed content. The buffer needs to be released by the receiver
     */
    ByteBuf encode(ByteBuf raw);

Need to be released.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll fix this.


CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer();
compositeByteBuf.addComponent(true, metadataByteBuf);
compositeByteBuf.addComponent(true, encodeByteBuf);

byte[] dataBytes = new byte[compositeByteBuf.readableBytes()];
compositeByteBuf.readBytes(dataBytes);
return dataBytes;
Copy link
Contributor

@315157973 315157973 Jul 29, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do these ByteBufs need to be released?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Release the compositeByteBuf will encounter error o.netty.util.IllegalReferenceCountException: refCnt: 0, decrement: 1.

}

public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws InvalidProtocolBufferException {
ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(data.length, data.length);
byteBuf.writeBytes(data);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems we can use wrappedBuffer to reduce one copy

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I'll fix this.

if (byteBuf.readableBytes() > 0 && byteBuf.readShort() == magicManagedLedgerInfoMetadata) {
try {
int metadataSize = byteBuf.readInt();
ManagedLedgerInfoMetadata metadata = ML_INFO_METADATA.get();
metadata.parseFrom(byteBuf, metadataSize);

long unpressedSize = metadata.getUnpressedSize();
ByteBuf decodeByteBuf = CompressionCodecProvider.getCompressionCodec(metadata.getCompressionType())
.decode(byteBuf, (int) unpressedSize);
return ManagedLedgerInfo.parseFrom(decodeByteBuf.array());
} catch (Exception e) {
log.error("Failed to parse managedLedgerInfo metadata, "
+ "fall back to parse managedLedgerInfo directly.", e);
byteBuf.resetReaderIndex();
return ManagedLedgerInfo.parseFrom(data);
}
} else {
return ManagedLedgerInfo.parseFrom(data);
}
}

}
@@ -0,0 +1,85 @@
/**
* 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 java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
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.Test;

/**
* ManagedLedgerInfo metadata test.
*/
@Slf4j
public class ManagedLedgerInfoMetadataTest {

@Test
public void testEncodeAndDecode() 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);
byte[] compressionBytes = metaStore.compressLedgerInfo(managedLedgerInfo, CompressionType.ZSTD);
log.info("Uncompressed data size: {}, compressed data size: {}",
managedLedgerInfo.getSerializedSize(), compressionBytes.length);

MLDataFormats.ManagedLedgerInfo info1 = metaStore.parseManagedLedgerInfo(compressionBytes);
MLDataFormats.ManagedLedgerInfo info2 = metaStore.parseManagedLedgerInfo(managedLedgerInfo.toByteArray());
Assert.assertEquals(info1, info2);
}

}
5 changes: 5 additions & 0 deletions pulsar-common/src/main/proto/PulsarApi.proto
Expand Up @@ -186,6 +186,11 @@ message BrokerEntryMetadata {
optional uint64 index = 2;
}

message ManagedLedgerInfoMetadata {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gaoran10 We should move it to MLDataFormats.proto? The PulsarApi.proto is used for the broker and client interaction.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I'll move it.

required CompressionType compressionType = 1;
required int32 unpressedSize = 2;
}

enum ServerError {
UnknownError = 0;
MetadataError = 1; // Error with ZK/metadata
Expand Down