Skip to content

Commit

Permalink
Merge pull request #377 from Aiven-Open/ivanyu/add-metadata-to-manifest
Browse files Browse the repository at this point in the history
Store remote log segment metadata
  • Loading branch information
AnatolyPopov authored Sep 27, 2023
2 parents a60c423 + f943941 commit 2cfdbbd
Show file tree
Hide file tree
Showing 15 changed files with 245 additions and 50 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
package io.aiven.kafka.tieredstorage;

import javax.crypto.Cipher;
import javax.crypto.SecretKey;
import javax.crypto.spec.GCMParameterSpec;
import javax.crypto.spec.SecretKeySpec;

Expand Down Expand Up @@ -55,8 +54,8 @@
import io.aiven.kafka.tieredstorage.chunkmanager.cache.DiskBasedChunkCache;
import io.aiven.kafka.tieredstorage.chunkmanager.cache.InMemoryChunkCache;
import io.aiven.kafka.tieredstorage.manifest.index.ChunkIndex;
import io.aiven.kafka.tieredstorage.manifest.serde.DataKeyDeserializer;
import io.aiven.kafka.tieredstorage.manifest.serde.DataKeySerializer;
import io.aiven.kafka.tieredstorage.manifest.serde.EncryptionSerdeModule;
import io.aiven.kafka.tieredstorage.manifest.serde.KafkaTypeSerdeModule;
import io.aiven.kafka.tieredstorage.metadata.SegmentCustomMetadataField;
import io.aiven.kafka.tieredstorage.metadata.SegmentCustomMetadataSerde;
import io.aiven.kafka.tieredstorage.security.AesEncryptionProvider;
Expand All @@ -66,7 +65,6 @@

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.github.luben.zstd.Zstd;
import org.assertj.core.api.InstanceOfAssertFactories;
import org.junit.jupiter.api.BeforeEach;
Expand Down Expand Up @@ -328,6 +326,7 @@ private void checkEncryption(final boolean compression) throws IOException {
// 3. The AAD is used.

final ObjectMapper objectMapper = new ObjectMapper();
objectMapper.registerModule(KafkaTypeSerdeModule.create());
final JsonNode manifest = objectMapper.readTree(new File(targetDir.toString(), TARGET_MANIFEST_FILE));

final String dataKeyText = manifest.get("encryption").get("dataKey").asText();
Expand All @@ -338,13 +337,8 @@ private void checkEncryption(final boolean compression) throws IOException {
final byte[] dataKey = rsaEncryptionProvider.decryptDataKey(
new EncryptedDataKey(KEY_ENCRYPTION_KEY_ID, encryptedDataKey));
final byte[] aad = manifest.get("encryption").get("aad").binaryValue();
objectMapper.registerModule(EncryptionSerdeModule.create(rsaEncryptionProvider));

final SimpleModule simpleModule = new SimpleModule();
simpleModule.addSerializer(SecretKey.class,
new DataKeySerializer(rsaEncryptionProvider::encryptDataKey));
simpleModule.addDeserializer(SecretKey.class,
new DataKeyDeserializer(rsaEncryptionProvider::decryptDataKey));
objectMapper.registerModule(simpleModule);
final ChunkIndex chunkIndex = objectMapper.treeToValue(manifest.get("chunkIndex"), ChunkIndex.class);

try (final InputStream originalInputStream = Files.newInputStream(logFilePath);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,6 @@

package io.aiven.kafka.tieredstorage;

import javax.crypto.SecretKey;

import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
Expand Down Expand Up @@ -52,8 +50,8 @@
import io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider;
import io.aiven.kafka.tieredstorage.manifest.SegmentManifestV1;
import io.aiven.kafka.tieredstorage.manifest.index.ChunkIndex;
import io.aiven.kafka.tieredstorage.manifest.serde.DataKeyDeserializer;
import io.aiven.kafka.tieredstorage.manifest.serde.DataKeySerializer;
import io.aiven.kafka.tieredstorage.manifest.serde.EncryptionSerdeModule;
import io.aiven.kafka.tieredstorage.manifest.serde.KafkaTypeSerdeModule;
import io.aiven.kafka.tieredstorage.metadata.SegmentCustomMetadataBuilder;
import io.aiven.kafka.tieredstorage.metadata.SegmentCustomMetadataField;
import io.aiven.kafka.tieredstorage.metadata.SegmentCustomMetadataSerde;
Expand Down Expand Up @@ -81,7 +79,6 @@
import io.aiven.kafka.tieredstorage.transform.TransformFinisher;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -180,13 +177,9 @@ void setStorage(final StorageBackend storage) {
private ObjectMapper getObjectMapper() {
final ObjectMapper objectMapper = new ObjectMapper();
objectMapper.registerModule(new Jdk8Module());
objectMapper.registerModule(KafkaTypeSerdeModule.create());
if (encryptionEnabled) {
final SimpleModule simpleModule = new SimpleModule();
simpleModule.addSerializer(SecretKey.class,
new DataKeySerializer(rsaEncryptionProvider::encryptDataKey));
simpleModule.addDeserializer(SecretKey.class,
new DataKeyDeserializer(rsaEncryptionProvider::decryptDataKey));
objectMapper.registerModule(simpleModule);
objectMapper.registerModule(EncryptionSerdeModule.create(rsaEncryptionProvider));
}
return objectMapper;
}
Expand Down Expand Up @@ -229,7 +222,7 @@ public Optional<CustomMetadata> copyLogSegmentData(final RemoteLogSegmentMetadat

final ChunkIndex chunkIndex = transformFinisher.chunkIndex();
final SegmentManifest segmentManifest =
new SegmentManifestV1(chunkIndex, requiresCompression, encryptionMetadata);
new SegmentManifestV1(chunkIndex, requiresCompression, encryptionMetadata, remoteLogSegmentMetadata);
uploadManifest(remoteLogSegmentMetadata, segmentManifest, customMetadataBuilder);

final InputStream offsetIndex = Files.newInputStream(logSegmentData.offsetIndex());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

import java.util.Optional;

import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;

import io.aiven.kafka.tieredstorage.manifest.index.ChunkIndex;

import com.fasterxml.jackson.annotation.JsonSubTypes;
Expand All @@ -38,4 +40,6 @@ public interface SegmentManifest {
boolean compression();

Optional<SegmentEncryptionMetadata> encryption();

RemoteLogSegmentMetadata remoteLogSegmentMetadata();
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import java.util.Objects;
import java.util.Optional;

import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;

import io.aiven.kafka.tieredstorage.manifest.index.ChunkIndex;

import com.fasterxml.jackson.annotation.JsonCreator;
Expand All @@ -29,6 +31,7 @@ public class SegmentManifestV1 implements SegmentManifest {
private final ChunkIndex chunkIndex;
private final boolean compression;
private final SegmentEncryptionMetadataV1 encryption;
private final RemoteLogSegmentMetadata remoteLogSegmentMetadata;

@JsonCreator
public SegmentManifestV1(@JsonProperty(value = "chunkIndex", required = true)
Expand All @@ -37,10 +40,19 @@ public SegmentManifestV1(@JsonProperty(value = "chunkIndex", required = true)
final boolean compression,
@JsonProperty("encryption")
final SegmentEncryptionMetadataV1 encryption) {
this(chunkIndex, compression, encryption, null);
}

public SegmentManifestV1(final ChunkIndex chunkIndex,
final boolean compression,
final SegmentEncryptionMetadataV1 encryption,
final RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
this.chunkIndex = Objects.requireNonNull(chunkIndex, "chunkIndex cannot be null");
this.compression = compression;

this.encryption = encryption;

this.remoteLogSegmentMetadata = remoteLogSegmentMetadata;
}

@Override
Expand All @@ -62,6 +74,13 @@ public Optional<SegmentEncryptionMetadata> encryption() {
return Optional.ofNullable(encryption);
}

@Override
// We don't need to deserialize it
@JsonProperty(value = "remoteLogSegmentMetadata", access = JsonProperty.Access.READ_ONLY)
public RemoteLogSegmentMetadata remoteLogSegmentMetadata() {
return remoteLogSegmentMetadata;
}

@Override
public boolean equals(final Object o) {
if (this == o) {
Expand All @@ -79,6 +98,7 @@ public boolean equals(final Object o) {
if (!chunkIndex.equals(that.chunkIndex)) {
return false;
}
// We don't want remoteLogSegmentMetadata to participate in hash code and equality checks.
return Objects.equals(encryption, that.encryption);
}

Expand All @@ -87,6 +107,7 @@ public int hashCode() {
int result = chunkIndex.hashCode();
result = 31 * result + (compression ? 1 : 0);
result = 31 * result + (encryption != null ? encryption.hashCode() : 0);
// We don't want remoteLogSegmentMetadata to participate in hash code and equality checks.
return result;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,10 @@
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;

public class DataKeyDeserializer extends StdDeserializer<SecretKey> {
class DataKeyDeserializer extends StdDeserializer<SecretKey> {
private final Function<EncryptedDataKey, byte[]> keyDecryptor;

public DataKeyDeserializer(final Function<EncryptedDataKey, byte[]> keyDecryptor) {
DataKeyDeserializer(final Function<EncryptedDataKey, byte[]> keyDecryptor) {
super(SecretKey.class);
this.keyDecryptor = Objects.requireNonNull(keyDecryptor, "keyDecryptor cannot be null");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,10 @@
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;

public class DataKeySerializer extends StdSerializer<SecretKey> {
class DataKeySerializer extends StdSerializer<SecretKey> {
private final Function<byte[], EncryptedDataKey> dataKeyEncryptor;

public DataKeySerializer(final Function<byte[], EncryptedDataKey> dataKeyEncryptor) {
DataKeySerializer(final Function<byte[], EncryptedDataKey> dataKeyEncryptor) {
super(SecretKey.class);
this.dataKeyEncryptor = Objects.requireNonNull(dataKeyEncryptor, "dataKeyEncryptor cannot be null");
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* Copyright 2023 Aiven Oy
*
* Licensed 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 io.aiven.kafka.tieredstorage.manifest.serde;

import javax.crypto.SecretKey;

import io.aiven.kafka.tieredstorage.security.RsaEncryptionProvider;

import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.module.SimpleModule;

public final class EncryptionSerdeModule {
public static Module create(final RsaEncryptionProvider rsaEncryptionProvider) {
final var module = new SimpleModule();

module.addSerializer(SecretKey.class,
new DataKeySerializer(rsaEncryptionProvider::encryptDataKey));
module.addDeserializer(SecretKey.class,
new DataKeyDeserializer(rsaEncryptionProvider::decryptDataKey));

return module;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Copyright 2023 Aiven Oy
*
* Licensed 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 io.aiven.kafka.tieredstorage.manifest.serde;

import java.io.IOException;
import java.util.Map;

import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;

import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyOrder;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;

public class KafkaTypeSerdeModule {
public static Module create() {
final var module = new SimpleModule();

module.addSerializer(Uuid.class, new UuidSerializer());

module.setMixInAnnotation(TopicPartition.class, TopicPartitionSerdeMixin.class);
module.setMixInAnnotation(TopicIdPartition.class, TopicIdPartitionSerdeMixin.class);
module.setMixInAnnotation(RemoteLogSegmentId.class, RemoteLogSegmentIdMixin.class);
module.setMixInAnnotation(RemoteLogSegmentMetadata.class, RemoteLogSegmentMetadataMixin.class);

return module;
}

private static class UuidSerializer extends StdSerializer<Uuid> {
UuidSerializer() {
super(Uuid.class);
}

@Override
public void serialize(final Uuid value,
final JsonGenerator gen,
final SerializerProvider provider) throws IOException {
gen.writeString(value.toString());
}
}

@JsonPropertyOrder({ "topic", "partition" })
private abstract static class TopicPartitionSerdeMixin {
@JsonProperty("topic")
public abstract String topic();

@JsonProperty("partition")
public abstract int partition();
}

@JsonPropertyOrder({ "topicId", "topicPartition" })
private abstract static class TopicIdPartitionSerdeMixin {
@JsonProperty("topicId")
public abstract Uuid topicId();

@JsonProperty("topicPartition")
public abstract TopicPartition topicPartition();
}

@JsonPropertyOrder({ "topicIdPartition", "id" })
private abstract static class RemoteLogSegmentIdMixin {
@JsonProperty("topicIdPartition")
abstract TopicIdPartition topicIdPartition();

@JsonProperty("id")
abstract Uuid id();
}

@JsonPropertyOrder({
"remoteLogSegmentId", "startOffset", "endOffset",
"maxTimestampMs", "brokerId", "eventTimestampMs", "segmentLeaderEpochs"
})
private abstract static class RemoteLogSegmentMetadataMixin {
@JsonProperty("remoteLogSegmentId")
public abstract RemoteLogSegmentId remoteLogSegmentId();

@JsonProperty("startOffset")
public abstract long startOffset();

@JsonProperty("endOffset")
public abstract long endOffset();

@JsonProperty("maxTimestampMs")
public abstract long maxTimestampMs();

@JsonProperty("brokerId")
public abstract int brokerId();

@JsonProperty("eventTimestampMs")
public abstract long eventTimestampMs();

@JsonProperty("segmentLeaderEpochs")
public abstract Map<Integer, Long> segmentLeaderEpochs();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -149,9 +149,9 @@ void metricsShouldBeReported(final String tags) throws RemoteStorageException, J
.isEqualTo(18.0 / METRIC_TIME_WINDOW_SEC);

assertThat(MBEAN_SERVER.getAttribute(metricName, "object-upload-bytes-total"))
.isEqualTo(657.0);
.isEqualTo(1575.0);
assertThat(MBEAN_SERVER.getAttribute(metricName, "object-upload-bytes-rate"))
.isEqualTo(657.0 / METRIC_TIME_WINDOW_SEC);
.isEqualTo(1575.0 / METRIC_TIME_WINDOW_SEC);

for (final var suffix : ObjectKey.Suffix.values()) {
final ObjectName storageMetricsName = ObjectName.getInstance(objectName + ",object-type=" + suffix.value);
Expand Down
Loading

0 comments on commit 2cfdbbd

Please sign in to comment.