Skip to content

Commit

Permalink
refactor: segment manifest builder
Browse files Browse the repository at this point in the history
Bunch of optional fields can be passed via builder
  • Loading branch information
jeqo committed Apr 8, 2024
1 parent cdffc7f commit d1259b0
Show file tree
Hide file tree
Showing 10 changed files with 234 additions and 39 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,6 @@
import io.aiven.kafka.tieredstorage.fetch.index.MemorySegmentIndexesCache;
import io.aiven.kafka.tieredstorage.fetch.index.SegmentIndexesCache;
import io.aiven.kafka.tieredstorage.manifest.SegmentEncryptionMetadata;
import io.aiven.kafka.tieredstorage.manifest.SegmentEncryptionMetadataV1;
import io.aiven.kafka.tieredstorage.manifest.SegmentIndex;
import io.aiven.kafka.tieredstorage.manifest.SegmentIndexesV1;
import io.aiven.kafka.tieredstorage.manifest.SegmentIndexesV1Builder;
Expand Down Expand Up @@ -481,19 +480,13 @@ void uploadManifest(final RemoteLogSegmentMetadata remoteLogSegmentMetadata,
final DataKeyAndAAD maybeEncryptionKey,
final SegmentCustomMetadataBuilder customMetadataBuilder
) throws StorageBackendException, IOException {
final SegmentEncryptionMetadataV1 maybeEncryptionMetadata;
final var segmentManifestBuilder = SegmentManifestV1.newBuilder(chunkIndex, segmentIndexes)
.withRlsm(remoteLogSegmentMetadata)
.withCompressionEnabled(requiresCompression);
if (maybeEncryptionKey != null) {
maybeEncryptionMetadata = new SegmentEncryptionMetadataV1(maybeEncryptionKey);
} else {
maybeEncryptionMetadata = null;
segmentManifestBuilder.withEncryptionKey(maybeEncryptionKey);
}
final SegmentManifest segmentManifest = new SegmentManifestV1(
chunkIndex,
segmentIndexes,
requiresCompression,
maybeEncryptionMetadata,
remoteLogSegmentMetadata
);
final SegmentManifest segmentManifest = segmentManifestBuilder.build();
final String manifest = mapper.writeValueAsString(segmentManifest);
final ObjectKey manifestObjectKey =
objectKeyFactory.key(remoteLogSegmentMetadata, ObjectKeyFactory.Suffix.MANIFEST);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;

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

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
Expand All @@ -44,7 +45,7 @@ public SegmentManifestV1(
this(chunkIndex, segmentIndexes, compression, encryption, null);
}

public SegmentManifestV1(final ChunkIndex chunkIndex,
private SegmentManifestV1(final ChunkIndex chunkIndex,
final SegmentIndexesV1 segmentIndexes,
final boolean compression,
final SegmentEncryptionMetadataV1 encryption,
Expand All @@ -58,6 +59,13 @@ public SegmentManifestV1(final ChunkIndex chunkIndex,
this.remoteLogSegmentMetadata = remoteLogSegmentMetadata;
}

public static Builder newBuilder(
final ChunkIndex chunkIndex,
final SegmentIndexesV1 segmentIndexes
) {
return new Builder(chunkIndex, segmentIndexes);
}

@Override
@JsonProperty("chunkIndex")
public ChunkIndex chunkIndex() {
Expand Down Expand Up @@ -129,4 +137,46 @@ public String toString() {
+ ", encryption=" + encryption
+ ")";
}

public static class Builder {
final ChunkIndex chunkIndex;
final SegmentIndexesV1 segmentIndexes;
boolean compression = false;
SegmentEncryptionMetadataV1 encryptionMetadata = null;
RemoteLogSegmentMetadata rlsm = null;

public Builder(
final ChunkIndex chunkIndex,
final SegmentIndexesV1 segmentIndexes
) {
this.chunkIndex = chunkIndex;
this.segmentIndexes = segmentIndexes;
}

public Builder withCompressionEnabled(final boolean requiresCompression) {
this.compression = requiresCompression;
return this;
}

public Builder withEncryptionMetadata(final SegmentEncryptionMetadataV1 encryptionMetadata) {
this.encryptionMetadata = Objects.requireNonNull(encryptionMetadata, "encryptionMetadata cannot be null");
return this;
}

public Builder withEncryptionKey(final DataKeyAndAAD dataKeyAndAAD) {
this.encryptionMetadata = new SegmentEncryptionMetadataV1(
Objects.requireNonNull(dataKeyAndAAD, "dataKeyAndAAD cannot be null")
);
return this;
}

public Builder withRlsm(final RemoteLogSegmentMetadata rlsm) {
this.rlsm = Objects.requireNonNull(rlsm, "remoteLogSegmentMetadata cannot be null");
return this;
}

public SegmentManifestV1 build() {
return new SegmentManifestV1(chunkIndex, segmentIndexes, compression, encryptionMetadata, rlsm);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,8 @@ class DefaultChunkManagerTest extends AesKeyAwareTest {
void testGetChunk() throws Exception {
final FixedSizeChunkIndex chunkIndex = new FixedSizeChunkIndex(10, 10, 10, 10);

final SegmentManifest manifest = new SegmentManifestV1(chunkIndex, SEGMENT_INDEXES, false, null, null);
final SegmentManifest manifest = SegmentManifestV1.newBuilder(chunkIndex, SEGMENT_INDEXES)
.build();
final ChunkManager chunkManager = new DefaultChunkManager(storage, null);
when(storage.fetch(OBJECT_KEY, chunkIndex.chunks().get(0).range()))
.thenReturn(new ByteArrayInputStream("0123456789".getBytes()));
Expand All @@ -88,7 +89,9 @@ void testGetChunkWithEncryption() throws Exception {
new ByteArrayInputStream(encrypted));

final var encryption = new SegmentEncryptionMetadataV1(dataKeyAndAAD.dataKey, dataKeyAndAAD.aad);
final var manifest = new SegmentManifestV1(chunkIndex, SEGMENT_INDEXES, false, encryption, null);
final var manifest = SegmentManifestV1.newBuilder(chunkIndex, SEGMENT_INDEXES)
.withEncryptionMetadata(encryption)
.build();
final ChunkManager chunkManager = new DefaultChunkManager(storage, aesEncryptionProvider);

assertThat(chunkManager.getChunk(OBJECT_KEY, manifest, 0)).hasBinaryContent(TEST_CHUNK_CONTENT);
Expand All @@ -108,7 +111,9 @@ void testGetChunkWithCompression() throws Exception {
when(storage.fetch(OBJECT_KEY, chunkIndex.chunks().get(0).range()))
.thenReturn(new ByteArrayInputStream(compressed));

final var manifest = new SegmentManifestV1(chunkIndex, SEGMENT_INDEXES, true, null, null);
final var manifest = SegmentManifestV1.newBuilder(chunkIndex, SEGMENT_INDEXES)
.withCompressionEnabled(true)
.build();
final ChunkManager chunkManager = new DefaultChunkManager(storage, null);

assertThat(chunkManager.getChunk(OBJECT_KEY, manifest, 0)).hasBinaryContent(TEST_CHUNK_CONTENT);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,8 @@ class FetchChunkEnumerationSourceInputStreamClosingTest {
.add(IndexType.LEADER_EPOCH, 1)
.add(IndexType.TRANSACTION, 1)
.build();
static final SegmentManifest SEGMENT_MANIFEST = new SegmentManifestV1(
CHUNK_INDEX, SEGMENT_INDEXES, false, null, null);
static final SegmentManifest SEGMENT_MANIFEST = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.build();

TestObjectFetcher fetcher;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ class FetchChunkEnumerationTest {
.add(IndexType.LEADER_EPOCH, 1)
.add(IndexType.TRANSACTION, 1)
.build();
final SegmentManifest manifest = new SegmentManifestV1(chunkIndex, segmentIndexesV1, false, null, null);
final SegmentManifest manifest = SegmentManifestV1.newBuilder(chunkIndex, segmentIndexesV1).build();

static final byte[] CHUNK_CONTENT = "0123456789".getBytes();
static final ObjectKey SEGMENT_KEY = new TestObjectKey("topic/segment");
Expand All @@ -64,7 +64,7 @@ class FetchChunkEnumerationTest {
@Test
void failsWhenLargerStartPosition() {
// Given
final SegmentManifest manifest = new SegmentManifestV1(chunkIndex, segmentIndexesV1, false, null, null);
final SegmentManifest manifest = SegmentManifestV1.newBuilder(chunkIndex, segmentIndexesV1).build();
// When
final int from = 1000;
final int to = from + 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,11 @@ class ChunkCacheTest {
.add(IndexType.TRANSACTION, 1)
.build();

private static final SegmentManifest SEGMENT_MANIFEST =
new SegmentManifestV1(FIXED_SIZE_CHUNK_INDEX, SEGMENT_INDEXES, false, null, null);
private static final SegmentManifest SEGMENT_MANIFEST = SegmentManifestV1.newBuilder(
FIXED_SIZE_CHUNK_INDEX,
SEGMENT_INDEXES
)
.build();
private static final String TEST_EXCEPTION_MESSAGE = "test_message";
private static final String SEGMENT_KEY = "topic/segment";
private static final ObjectKey SEGMENT_OBJECT_KEY = () -> SEGMENT_KEY;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,16 +56,16 @@ class DiskChunkCacheMetricsTest {
TimeUnit.SECONDS.convert(new MetricConfig().timeWindowMs(), TimeUnit.MILLISECONDS);

static final SegmentManifest SEGMENT_MANIFEST =
new SegmentManifestV1(
new FixedSizeChunkIndex(10, 30, 10, 10),
SegmentIndexesV1.builder()
.add(RemoteStorageManager.IndexType.OFFSET, 1)
.add(RemoteStorageManager.IndexType.TIMESTAMP, 1)
.add(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, 1)
.add(RemoteStorageManager.IndexType.LEADER_EPOCH, 1)
.add(RemoteStorageManager.IndexType.TRANSACTION, 1)
.build(),
false, null, null);
SegmentManifestV1.newBuilder(
new FixedSizeChunkIndex(10, 30, 10, 10),
SegmentIndexesV1.builder()
.add(RemoteStorageManager.IndexType.OFFSET, 1)
.add(RemoteStorageManager.IndexType.TIMESTAMP, 1)
.add(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, 1)
.add(RemoteStorageManager.IndexType.LEADER_EPOCH, 1)
.add(RemoteStorageManager.IndexType.TRANSACTION, 1)
.build())
.build();

static final ObjectKey OBJECT_KEY_PATH = () -> "topic/segment";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ void shouldReturnAndCache() throws StorageBackendException, IOException {
when(storage.fetch(key))
.thenReturn(new ByteArrayInputStream(MANIFEST.getBytes()));
final var chunkIndex = new FixedSizeChunkIndex(100, 1000, 110, 110);
final var expectedManifest = new SegmentManifestV1(chunkIndex, SEGMENT_INDEXES, false, null, null);
final var expectedManifest = SegmentManifestV1.newBuilder(chunkIndex, SEGMENT_INDEXES).build();
assertThat(provider.get(key)).isEqualTo(expectedManifest);
verify(storage).fetch(key);
assertThat(provider.get(key)).isEqualTo(expectedManifest);
Expand Down Expand Up @@ -155,7 +155,7 @@ void shouldNotPoisonCacheWithFailedFutures()
.hasMessage("test");

final var chunkIndex = new FixedSizeChunkIndex(100, 1000, 110, 110);
final var expectedManifest = new SegmentManifestV1(chunkIndex, SEGMENT_INDEXES, false, null, null);
final var expectedManifest = SegmentManifestV1.newBuilder(chunkIndex, SEGMENT_INDEXES).build();

await().atMost(Duration.ofMillis(50))
.pollInterval(Duration.ofMillis(5))
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
/*
* Copyright 2024 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;

import io.aiven.kafka.tieredstorage.manifest.index.FixedSizeChunkIndex;
import io.aiven.kafka.tieredstorage.security.DataKeyAndAAD;
import org.apache.kafka.common.TopicIdPartition;
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 org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
import org.junit.jupiter.api.Test;

import javax.crypto.SecretKey;
import javax.crypto.spec.SecretKeySpec;
import java.util.Map;

import static org.assertj.core.api.Assertions.assertThat;

class SegmentManifestV1BuilderTest {
static final FixedSizeChunkIndex CHUNK_INDEX =
new FixedSizeChunkIndex(100, 1000, 110, 110);
static final SecretKey DATA_KEY = new SecretKeySpec(new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, "AES");
static final byte[] AAD = {10, 11, 12, 13};
static final SegmentIndexesV1 SEGMENT_INDEXES = new SegmentIndexesV1Builder()
.add(RemoteStorageManager.IndexType.OFFSET, 1)
.add(RemoteStorageManager.IndexType.TIMESTAMP, 1)
.add(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, 1)
.add(RemoteStorageManager.IndexType.LEADER_EPOCH, 1)
.add(RemoteStorageManager.IndexType.TRANSACTION, 1)
.build();
static final RemoteLogSegmentMetadata REMOTE_LOG_SEGMENT_METADATA = new RemoteLogSegmentMetadata(
new RemoteLogSegmentId(
new TopicIdPartition(Uuid.fromString("lZ6vvmajTWKDBUTV6SQAtQ"), 42, "topic1"),
Uuid.fromString("adh9f8BMS4anaUnD8KWfWg")
),
0,
1000L,
1000000000L,
2,
2000000000L,
100500,
Map.of(0, 100L, 1, 200L, 2, 300L)
);

@Test
void minimal() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES).build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isFalse();
assertThat(manifest.encryption()).isEmpty();
assertThat(manifest.remoteLogSegmentMetadata()).isNull();
}

@Test
void withRlsm() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.withRlsm(REMOTE_LOG_SEGMENT_METADATA)
.build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isFalse();
assertThat(manifest.encryption()).isEmpty();
assertThat(manifest.remoteLogSegmentMetadata()).isEqualTo(REMOTE_LOG_SEGMENT_METADATA);
}

@Test
void withCompressionEnabled() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.withCompressionEnabled(true)
.build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isTrue();
assertThat(manifest.encryption()).isEmpty();
assertThat(manifest.remoteLogSegmentMetadata()).isNull();
}

@Test
void withCompressionDisabled() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.withCompressionEnabled(false)
.build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isFalse();
assertThat(manifest.encryption()).isEmpty();
assertThat(manifest.remoteLogSegmentMetadata()).isNull();
}

@Test
void withEncryptionKey() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.withEncryptionKey(new DataKeyAndAAD(DATA_KEY, AAD))
.build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isFalse();
assertThat(manifest.encryption()).isPresent();
manifest.encryption().ifPresent(segmentEncryptionMetadata -> {
assertThat(segmentEncryptionMetadata.dataKey()).isEqualTo(DATA_KEY);
assertThat(segmentEncryptionMetadata.aad()).isEqualTo(AAD);
});
assertThat(manifest.remoteLogSegmentMetadata()).isNull();
}

@Test
void full() {
final var manifest = SegmentManifestV1.newBuilder(CHUNK_INDEX, SEGMENT_INDEXES)
.withCompressionEnabled(true)
.withEncryptionKey(new DataKeyAndAAD(DATA_KEY, AAD))
.withRlsm(REMOTE_LOG_SEGMENT_METADATA)
.build();
assertThat(manifest.chunkIndex()).isEqualTo(CHUNK_INDEX);
assertThat(manifest.segmentIndexes()).isEqualTo(SEGMENT_INDEXES);
assertThat(manifest.compression()).isTrue();
assertThat(manifest.encryption()).isPresent();
manifest.encryption().ifPresent(segmentEncryptionMetadata -> {
assertThat(segmentEncryptionMetadata.dataKey()).isEqualTo(DATA_KEY);
assertThat(segmentEncryptionMetadata.aad()).isEqualTo(AAD);
});
assertThat(manifest.remoteLogSegmentMetadata()).isEqualTo(REMOTE_LOG_SEGMENT_METADATA);
}
}
Loading

0 comments on commit d1259b0

Please sign in to comment.