Skip to content

Commit 477d487

Browse files
committed
refactor: decouple chunking disabling from file size
Currently, TransformFinisher piggybacks on original file size to disable chunking. This workaround was introduced when enabling transformations for individual indexes; but this logic has been recently replaced when concatenating indexes. Now chunk indexes are used, and chunking is disabled not by nullifying the chunk index but by setting a chunk size equal to the transformed size. By introducing a new flag "chunkingEnabled", the transformer logic can be cleaned up. This is needed for further changes in the transformer as we look into uploading directly from files.
1 parent dc45dda commit 477d487

File tree

5 files changed

+111
-71
lines changed

5 files changed

+111
-71
lines changed

core/src/main/java/io/aiven/kafka/tieredstorage/RemoteStorageManager.java

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -306,10 +306,11 @@ ChunkIndex uploadSegmentLog(
306306

307307
try (final var logSegmentInputStream = Files.newInputStream(logSegmentData.logSegment())) {
308308
final var transformEnum = transformation(logSegmentInputStream, requiresCompression, maybeEncryptionKey);
309-
final var transformFinisher = new TransformFinisher(
310-
transformEnum,
311-
remoteLogSegmentMetadata.segmentSizeInBytes()
312-
);
309+
final var transformFinisher = TransformFinisher.newBuilder(
310+
transformEnum,
311+
remoteLogSegmentMetadata.segmentSizeInBytes()
312+
)
313+
.build();
313314

314315
try (final var sis = transformFinisher.toInputStream()) {
315316
final var bytes = uploader.upload(sis, fileKey);
@@ -455,7 +456,9 @@ InputStream transformIndex(final IndexType indexType,
455456
() -> aesEncryptionProvider.encryptionCipher(maybeEncryptionKey)
456457
);
457458
}
458-
final var transformFinisher = new TransformFinisher(transformEnum, size);
459+
final var transformFinisher = TransformFinisher.newBuilder(transformEnum, size)
460+
.withChunkingDisabled()
461+
.build();
459462
final var inputStream = transformFinisher.nextElement();
460463
segmentIndexBuilder.add(indexType, singleChunk(transformFinisher.chunkIndex()).range().size());
461464
return inputStream;

core/src/main/java/io/aiven/kafka/tieredstorage/transform/BaseTransformChunkEnumeration.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -32,12 +32,6 @@ public class BaseTransformChunkEnumeration implements TransformChunkEnumeration
3232

3333
private byte[] chunk = null;
3434

35-
public BaseTransformChunkEnumeration(final InputStream inputStream) {
36-
this.inputStream = Objects.requireNonNull(inputStream, "inputStream cannot be null");
37-
38-
this.originalChunkSize = 0;
39-
}
40-
4135
public BaseTransformChunkEnumeration(final InputStream inputStream,
4236
final int originalChunkSize) {
4337
this.inputStream = Objects.requireNonNull(inputStream, "inputStream cannot be null");

core/src/main/java/io/aiven/kafka/tieredstorage/transform/TransformFinisher.java

Lines changed: 56 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,6 @@
2727
import io.aiven.kafka.tieredstorage.manifest.index.FixedSizeChunkIndexBuilder;
2828
import io.aiven.kafka.tieredstorage.manifest.index.VariableSizeChunkIndexBuilder;
2929

30-
// TODO test transforms and detransforms with property-based tests
31-
3230
/**
3331
* The transformation finisher.
3432
*
@@ -40,30 +38,40 @@
4038
public class TransformFinisher implements Enumeration<InputStream> {
4139
private final TransformChunkEnumeration inner;
4240
private final AbstractChunkIndexBuilder chunkIndexBuilder;
43-
private final int originalFileSize;
4441
private ChunkIndex chunkIndex = null;
4542

46-
public TransformFinisher(final TransformChunkEnumeration inner) {
47-
this(inner, 0);
43+
public static Builder newBuilder(final TransformChunkEnumeration inner, final int originalFileSize) {
44+
return new Builder(inner, originalFileSize);
4845
}
4946

50-
public TransformFinisher(final TransformChunkEnumeration inner, final int originalFileSize) {
47+
private TransformFinisher(
48+
final TransformChunkEnumeration inner,
49+
final boolean chunkingEnabled,
50+
final int originalFileSize
51+
) {
5152
this.inner = Objects.requireNonNull(inner, "inner cannot be null");
52-
this.originalFileSize = originalFileSize;
5353

54-
if (originalFileSize < 0) {
55-
throw new IllegalArgumentException(
56-
"originalFileSize must be non-negative, " + originalFileSize + " given");
57-
}
54+
final int originalChunkSize = chunkingEnabled ? inner.originalChunkSize() : Integer.MAX_VALUE;
55+
this.chunkIndexBuilder = chunkIndexBuilder(inner, originalChunkSize, originalFileSize);
56+
}
5857

58+
private static AbstractChunkIndexBuilder chunkIndexBuilder(
59+
final TransformChunkEnumeration inner,
60+
final int originalChunkSize,
61+
final int originalFileSize
62+
) {
5963
final Integer transformedChunkSize = inner.transformedChunkSize();
60-
if (originalFileSize == 0) {
61-
this.chunkIndexBuilder = null;
62-
} else if (transformedChunkSize == null) {
63-
this.chunkIndexBuilder = new VariableSizeChunkIndexBuilder(inner.originalChunkSize(), originalFileSize);
64+
if (transformedChunkSize == null) {
65+
return new VariableSizeChunkIndexBuilder(
66+
originalChunkSize,
67+
originalFileSize
68+
);
6469
} else {
65-
this.chunkIndexBuilder = new FixedSizeChunkIndexBuilder(
66-
inner.originalChunkSize(), originalFileSize, transformedChunkSize);
70+
return new FixedSizeChunkIndexBuilder(
71+
originalChunkSize,
72+
originalFileSize,
73+
transformedChunkSize
74+
);
6775
}
6876
}
6977

@@ -75,19 +83,17 @@ public boolean hasMoreElements() {
7583
@Override
7684
public InputStream nextElement() {
7785
final var chunk = inner.nextElement();
78-
if (chunkIndexBuilder != null) {
79-
if (hasMoreElements()) {
80-
this.chunkIndexBuilder.addChunk(chunk.length);
81-
} else {
82-
this.chunkIndex = this.chunkIndexBuilder.finish(chunk.length);
83-
}
86+
if (hasMoreElements()) {
87+
this.chunkIndexBuilder.addChunk(chunk.length);
88+
} else {
89+
this.chunkIndex = this.chunkIndexBuilder.finish(chunk.length);
8490
}
8591

8692
return new ByteArrayInputStream(chunk);
8793
}
8894

8995
public ChunkIndex chunkIndex() {
90-
if (chunkIndex == null && originalFileSize > 0) {
96+
if (chunkIndex == null) {
9197
throw new IllegalStateException("Chunk index was not built, was finisher used?");
9298
}
9399
return this.chunkIndex;
@@ -96,4 +102,30 @@ public ChunkIndex chunkIndex() {
96102
public InputStream toInputStream() {
97103
return new SequenceInputStream(this);
98104
}
105+
106+
public static class Builder {
107+
final TransformChunkEnumeration inner;
108+
final Integer originalFileSize;
109+
boolean chunkingEnabled = true;
110+
111+
public Builder(final TransformChunkEnumeration inner, final int originalFileSize) {
112+
this.inner = inner;
113+
114+
if (originalFileSize < 0) {
115+
throw new IllegalArgumentException(
116+
"originalFileSize must be non-negative, " + originalFileSize + " given");
117+
}
118+
119+
this.originalFileSize = originalFileSize;
120+
}
121+
122+
public Builder withChunkingDisabled() {
123+
this.chunkingEnabled = false;
124+
return this;
125+
}
126+
127+
public TransformFinisher build() {
128+
return new TransformFinisher(inner, chunkingEnabled, originalFileSize);
129+
}
130+
}
99131
}

core/src/test/java/io/aiven/kafka/tieredstorage/transform/TransformFinisherTest.java

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -42,22 +42,23 @@ class TransformFinisherTest {
4242

4343
@Test
4444
void getIndexBeforeUsing() {
45-
final TransformFinisher finisher = new TransformFinisher(new FakeDataEnumerator(3), 7);
45+
final TransformFinisher finisher = TransformFinisher.newBuilder(new FakeDataEnumerator(3), 7)
46+
.build();
4647
assertThatThrownBy(() -> finisher.chunkIndex())
4748
.isInstanceOf(IllegalStateException.class)
4849
.hasMessage("Chunk index was not built, was finisher used?");
4950
}
5051

5152
@Test
5253
void nullInnerEnumeration() {
53-
assertThatThrownBy(() -> new TransformFinisher(null, 100))
54+
assertThatThrownBy(() -> TransformFinisher.newBuilder(null, 100).build())
5455
.isInstanceOf(NullPointerException.class)
5556
.hasMessage("inner cannot be null");
5657
}
5758

5859
@Test
5960
void negativeOriginalFileSize() {
60-
assertThatThrownBy(() -> new TransformFinisher(inner, -1))
61+
assertThatThrownBy(() -> TransformFinisher.newBuilder(inner, -1).build())
6162
.isInstanceOf(IllegalArgumentException.class)
6263
.hasMessage("originalFileSize must be non-negative, -1 given");
6364
}
@@ -66,7 +67,8 @@ void negativeOriginalFileSize() {
6667
@MethodSource("provideForBuildIndexAndReturnCorrectInputStreams")
6768
void buildIndexAndReturnCorrectInputStreams(final Integer transformedChunkSize,
6869
final Class<ChunkIndex> indexType) throws IOException {
69-
final TransformFinisher finisher = new TransformFinisher(new FakeDataEnumerator(transformedChunkSize), 7);
70+
final TransformFinisher finisher = TransformFinisher.newBuilder(new FakeDataEnumerator(transformedChunkSize), 7)
71+
.build();
7072
assertThat(finisher.hasMoreElements()).isTrue();
7173
assertThat(finisher.nextElement().readAllBytes()).isEqualTo(new byte[] {0, 1, 2});
7274
assertThat(finisher.hasMoreElements()).isTrue();

core/src/test/java/io/aiven/kafka/tieredstorage/transform/TransformsEndToEndTest.java

Lines changed: 41 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -70,39 +70,48 @@ void compressionAndEncryption(final int chunkSize) throws IOException {
7070

7171
private void test(final int chunkSize, final boolean compression, final boolean encryption) throws IOException {
7272
// Transform.
73-
TransformChunkEnumeration transformEnum = new BaseTransformChunkEnumeration(
74-
new ByteArrayInputStream(original), chunkSize);
75-
if (compression) {
76-
transformEnum = new CompressionChunkEnumeration(transformEnum);
77-
}
78-
if (encryption) {
79-
transformEnum = new EncryptionChunkEnumeration(transformEnum, AesKeyAwareTest::encryptionCipherSupplier);
80-
}
81-
final var transformFinisher = chunkSize == 0
82-
? new TransformFinisher(transformEnum)
83-
: new TransformFinisher(transformEnum, ORIGINAL_SIZE);
84-
final byte[] uploadedData;
85-
final ChunkIndex chunkIndex;
86-
try (final var sis = transformFinisher.toInputStream()) {
87-
uploadedData = sis.readAllBytes();
88-
chunkIndex = transformFinisher.chunkIndex();
89-
}
73+
try (final var inputStream = new ByteArrayInputStream(original)) {
74+
TransformChunkEnumeration transformEnum = new BaseTransformChunkEnumeration(inputStream, chunkSize);
75+
if (compression) {
76+
transformEnum = new CompressionChunkEnumeration(transformEnum);
77+
}
78+
if (encryption) {
79+
transformEnum = new EncryptionChunkEnumeration(
80+
transformEnum,
81+
AesKeyAwareTest::encryptionCipherSupplier
82+
);
83+
}
84+
final var transformBuilder = TransformFinisher.newBuilder(transformEnum, ORIGINAL_SIZE);
85+
if (chunkSize == 0) {
86+
transformBuilder.withChunkingDisabled();
87+
}
88+
final var transformFinisher = transformBuilder.build();
89+
final byte[] uploadedData;
90+
final ChunkIndex chunkIndex;
91+
try (final var sis = transformFinisher.toInputStream()) {
92+
uploadedData = sis.readAllBytes();
93+
chunkIndex = transformFinisher.chunkIndex();
94+
}
9095

91-
// Detransform.
92-
DetransformChunkEnumeration detransformEnum = chunkIndex == null
93-
? new BaseDetransformChunkEnumeration(new ByteArrayInputStream(uploadedData))
94-
: new BaseDetransformChunkEnumeration(new ByteArrayInputStream(uploadedData), chunkIndex.chunks());
95-
if (encryption) {
96-
detransformEnum = new DecryptionChunkEnumeration(
97-
detransformEnum, ivSize, AesKeyAwareTest::decryptionCipherSupplier);
98-
}
99-
if (compression) {
100-
detransformEnum = new DecompressionChunkEnumeration(detransformEnum);
101-
}
102-
final var detransformFinisher = new DetransformFinisher(detransformEnum);
103-
try (final var sis = detransformFinisher.toInputStream()) {
104-
final byte[] downloaded = sis.readAllBytes();
105-
assertThat(downloaded).isEqualTo(original);
96+
// Detransform.
97+
try (final var uploadedStream = new ByteArrayInputStream(uploadedData)) {
98+
DetransformChunkEnumeration detransformEnum = new BaseDetransformChunkEnumeration(
99+
uploadedStream,
100+
chunkIndex.chunks()
101+
);
102+
if (encryption) {
103+
detransformEnum = new DecryptionChunkEnumeration(
104+
detransformEnum, ivSize, AesKeyAwareTest::decryptionCipherSupplier);
105+
}
106+
if (compression) {
107+
detransformEnum = new DecompressionChunkEnumeration(detransformEnum);
108+
}
109+
final var detransformFinisher = new DetransformFinisher(detransformEnum);
110+
try (final var sis = detransformFinisher.toInputStream()) {
111+
final byte[] downloaded = sis.readAllBytes();
112+
assertThat(downloaded).isEqualTo(original);
113+
}
114+
}
106115
}
107116
}
108117
}

0 commit comments

Comments
 (0)