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 2 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 @@ -75,4 +75,9 @@ public class ManagedLedgerFactoryConfig {
* cluster name for prometheus stats
*/
private String clusterName;

/**
* ManagedLedgerInfo compression type. If the compression type is null or invalid, don't compress data.
*/
private String managedLedgerInfoCompressionType = null;
Copy link
Contributor

Choose a reason for hiding this comment

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

We'd better use enum type to strict compression type info specific type instead of string. I'm not sure whether it's easy to implement.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Using the configuration managedLedgerInfoCompressionType of the ServiceConfiguration to set this field, maybe we could convert String to ComprssionType in MetaStoreImpl.

}
Expand Up @@ -172,7 +172,7 @@ private ManagedLedgerFactoryImpl(MetadataStore metadataStore,
this.bookkeeperFactory = bookKeeperGroupFactory;
this.isBookkeeperManaged = isBookkeeperManaged;
this.metadataStore = metadataStore;
this.store = new MetaStoreImpl(metadataStore, scheduledExecutor);
this.store = new MetaStoreImpl(metadataStore, scheduledExecutor, config.getManagedLedgerInfoCompressionType());
this.config = config;
this.mbean = new ManagedLedgerFactoryMBeanImpl(this);
this.entryCacheManager = new EntryCacheManager(this);
Expand Down
Expand Up @@ -25,15 +25,21 @@
import java.util.Optional;
import java.util.concurrent.CompletionException;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
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;
Expand All @@ -47,11 +53,27 @@ public class MetaStoreImpl implements MetaStore {
private final MetadataStore store;
private final OrderedExecutor executor;

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

Choose a reason for hiding this comment

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

private?

Copy link
Contributor

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

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 change this.

private CompressionType compressionType = null;
Copy link
Contributor

Choose a reason for hiding this comment

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

final ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's better.

Copy link
Contributor

Choose a reason for hiding this comment

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

initiate to be None instead of null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's better.


public MetaStoreImpl(MetadataStore store, OrderedExecutor executor) {
this.store = store;
this.executor = executor;
}

public MetaStoreImpl(MetadataStore store, OrderedExecutor executor, String compressionType) {
this.store = store;
this.executor = executor;
if (compressionType != null) {
try {
this.compressionType = CompressionType.valueOf(compressionType);
} catch (Exception e) {
log.warn("Failed to get compression type {}, disable managedLedgerInfo compression, error msg: {}.",
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

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 throw the exception.

compressionType, e.getMessage());
}
}
}

@Override
public void getManagedLedgerInfo(String ledgerName, boolean createIfMissing,
MetaStoreCallback<ManagedLedgerInfo> callback) {
Expand All @@ -62,7 +84,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 +123,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))));
Expand Down Expand Up @@ -264,4 +285,70 @@ 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();
}
byte[] originalBytes = managedLedgerInfo.toByteArray();
MLDataFormats.ManagedLedgerInfoMetadata mlInfoMetadata = MLDataFormats.ManagedLedgerInfoMetadata.newBuilder()
.setCompressionType(compressionType.name())
.setUnpressedSize(originalBytes.length)
.build();
ByteBuf 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());

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() == MAGIC_MANAGED_LEDGER_INFO_METADATA) {
try {
int metadataSize = byteBuf.readInt();
byte[] metadataBytes = new byte[metadataSize];
byteBuf.readBytes(metadataBytes);
MLDataFormats.ManagedLedgerInfoMetadata metadata =
MLDataFormats.ManagedLedgerInfoMetadata.parseFrom(metadataBytes);

long unpressedSize = metadata.getUnpressedSize();
ByteBuf decodeByteBuf = CompressionCodecProvider.getCompressionCodec(
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need a release here ?

Copy link
Contributor

Choose a reason for hiding this comment

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

/**
     * Decompress a buffer.
     *
     * <p>The buffer needs to have been compressed with the matching Encoder.
     *
     * @param encoded
     *            the compressed content
     * @param uncompressedSize
     *            the size of the original content
     * @return a ByteBuf with the compressed content. The buffer needs to be released by the receiver
     * @throws IOException
     *             if the decompression fails
     */
    ByteBuf decode(ByteBuf encoded, int uncompressedSize) throws IOException;

we need to release the ByteBuf

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.

CompressionType.valueOf(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);
return ManagedLedgerInfo.parseFrom(data);
} finally {
byteBuf.release();
}
} else {
try {
return ManagedLedgerInfo.parseFrom(data);
} finally {
byteBuf.release();
}
}
}

}
5 changes: 5 additions & 0 deletions managed-ledger/src/main/proto/MLDataFormats.proto
Expand Up @@ -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;
Copy link
Contributor

Choose a reason for hiding this comment

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

Use compression enum type instead of String?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The CompressionType in the PulsarApi.proto, if we want to use the enum CompressionType, we need import PulsarApi.proto, maybe we could use string type here?

Copy link
Contributor

Choose a reason for hiding this comment

The 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 int and use the other enum type

Copy link
Contributor

Choose a reason for hiding this comment

The 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;
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
required int32 unpressedSize = 2;
required int32 uncompressedSize = 2;

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.

}
@@ -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(), "");
}

}
Expand Up @@ -1601,6 +1601,11 @@ public class ServiceConfiguration implements PulsarConfiguration {
private String managedLedgerDataReadPriority = OffloadedReadPriority.TIERED_STORAGE_FIRST
.getValue();

@FieldContext(category = CATEGORY_STORAGE_ML,
doc = "ManagedLedgerInfo compression type, option values (LZ4, ZLIB, ZSTD, SNAPPY). \n"
Copy link
Contributor

Choose a reason for hiding this comment

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

add NONE compression value type to doc, and default value set to be NONE instead of null?

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 fix this.

+ "If not set value or value is invalid, then save the ManagedLedgerInfo bytes data directly.")
private String managedLedgerInfoCompressionType = null;

/*** --- Load balancer --- ****/
@FieldContext(
category = CATEGORY_LOAD_BALANCER,
Expand Down
Expand Up @@ -69,6 +69,7 @@ public void initialize(ServiceConfiguration conf, MetadataStore metadataStore,
conf.getManagedLedgerPrometheusStatsLatencyRolloverSeconds());
managedLedgerFactoryConfig.setTraceTaskExecution(conf.isManagedLedgerTraceTaskExecution());
managedLedgerFactoryConfig.setCursorPositionFlushSeconds(conf.getManagedLedgerCursorPositionFlushSeconds());
managedLedgerFactoryConfig.setManagedLedgerInfoCompressionType(conf.getManagedLedgerInfoCompressionType());

Configuration configuration = new ClientConfiguration();
if (conf.isBookkeeperClientExposeStatsToPrometheus()) {
Expand Down