Skip to content

Commit a51bcde

Browse files
committed
fix some
1 parent a0c0bb7 commit a51bcde

File tree

1 file changed

+54
-34
lines changed

1 file changed

+54
-34
lines changed

storage/src/test/java/org/apache/kafka/storage/internals/log/UnifiedLogTest.java

Lines changed: 54 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@
2424
import org.apache.kafka.common.record.RecordBatch;
2525
import org.apache.kafka.common.record.SimpleRecord;
2626
import org.apache.kafka.common.test.TestUtils;
27-
import org.apache.kafka.common.utils.PrimitiveRef;
2827
import org.apache.kafka.common.utils.Utils;
2928
import org.apache.kafka.coordinator.transaction.TransactionLogConfig;
3029
import org.apache.kafka.server.storage.log.FetchIsolation;
@@ -44,6 +43,7 @@
4443
import java.util.Optional;
4544
import java.util.Properties;
4645
import java.util.concurrent.ConcurrentHashMap;
46+
import java.util.function.Function;
4747
import java.util.function.Supplier;
4848

4949
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -109,16 +109,20 @@ public void followerShouldSaveEpochInformationFromReplicatedMessagesToTheEpochCa
109109
.mapToObj(id -> new SimpleRecord(String.valueOf(id).getBytes()))
110110
.toArray(SimpleRecord[]::new);
111111

112+
Function<Integer, MemoryRecords> recordsForEpoch = i -> {
113+
MemoryRecords recs = MemoryRecords.withRecords(messageIds[i], Compression.NONE, records[i]);
114+
recs.batches().forEach(record -> {
115+
record.setPartitionLeaderEpoch(42);
116+
record.setLastOffset(i);
117+
});
118+
return recs;
119+
};
120+
121+
// Given each message has an offset & epoch, as msgs from leader would
112122
try (UnifiedLog log = createLog(logDir, new LogConfig(new Properties()))) {
113123
// Given each message has an offset & epoch, as msgs from leader would
114124
for (int i = 0; i < records.length; i++) {
115-
long finalI = i;
116-
MemoryRecords recordsForEpoch = MemoryRecords.withRecords(messageIds[i], Compression.NONE, records[i]);
117-
recordsForEpoch.batches().forEach(batch -> {
118-
batch.setPartitionLeaderEpoch(42);
119-
batch.setLastOffset(finalI);
120-
});
121-
appendAsFollower(log, recordsForEpoch, i);
125+
log.appendAsFollower(recordsForEpoch.apply(i), i);
122126
}
123127

124128
assertEquals(Optional.of(42), log.latestEpoch());
@@ -154,17 +158,17 @@ public void shouldTruncateLeaderEpochsWhenDeletingSegments() throws IOException
154158

155159
@Test
156160
public void shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments() throws IOException {
157-
MemoryRecords records = TestUtils.singletonRecords("test".getBytes());
161+
Supplier<MemoryRecords> records = () -> TestUtils.singletonRecords("test".getBytes());
158162
LogConfig config = new LogTestUtils.LogConfigBuilder()
159-
.withSegmentBytes(records.sizeInBytes() * 5)
160-
.withRetentionBytes(records.sizeInBytes() * 10L)
163+
.withSegmentBytes(records.get().sizeInBytes() * 5)
164+
.withRetentionBytes(records.get().sizeInBytes() * 10L)
161165
.build();
162166

163167
log = createLog(logDir, config);
164168
LeaderEpochFileCache cache = epochCache(log);
165169

166170
for (int i = 0; i < 15; i++) {
167-
log.appendAsLeader(records, 0);
171+
log.appendAsLeader(records.get(), 0);
168172
}
169173

170174
// Given epochs
@@ -214,15 +218,16 @@ public void shouldTruncateLeaderEpochCheckpointFileWhenTruncatingLog() throws IO
214218

215219
@Test
216220
public void shouldDeleteSizeBasedSegments() throws IOException {
217-
MemoryRecords records = TestUtils.singletonRecords("test".getBytes());
221+
Supplier<MemoryRecords> records = () -> TestUtils.singletonRecords("test".getBytes());
218222
LogConfig config = new LogTestUtils.LogConfigBuilder()
219-
.withSegmentBytes(1024 * 1024 * 5)
223+
.withSegmentBytes(records.get().sizeInBytes() * 5)
224+
.withRetentionBytes(records.get().sizeInBytes() * 10L)
220225
.build();
221226
log = createLog(logDir, config);
222227

223228
// append some messages to create some segments
224229
for (int i = 0; i < 15; i++) {
225-
log.appendAsLeader(records, 0);
230+
log.appendAsLeader(records.get(), 0);
226231
}
227232

228233
log.updateHighWatermark(log.logEndOffset());
@@ -232,17 +237,17 @@ public void shouldDeleteSizeBasedSegments() throws IOException {
232237

233238
@Test
234239
public void shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() throws IOException {
235-
MemoryRecords records = TestUtils.singletonRecords("test".getBytes());
240+
Supplier<MemoryRecords> records = () -> TestUtils.singletonRecords("test".getBytes());
236241
LogConfig config = new LogTestUtils.LogConfigBuilder()
237-
.withSegmentBytes(records.sizeInBytes() * 5)
238-
.withRetentionBytes(records.sizeInBytes() * 15L)
242+
.withSegmentBytes(records.get().sizeInBytes() * 5)
243+
.withRetentionBytes(records.get().sizeInBytes() * 15L)
239244
.build();
240245

241246
log = createLog(logDir, config);
242247

243248
// append some messages to create some segments
244249
for (int i = 0; i < 15; i++) {
245-
log.appendAsLeader(records, 0);
250+
log.appendAsLeader(records.get(), 0);
246251
}
247252

248253
log.updateHighWatermark(log.logEndOffset());
@@ -252,15 +257,15 @@ public void shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() throws IOEx
252257

253258
@Test
254259
public void shouldDeleteTimeBasedSegmentsReadyToBeDeleted() throws IOException {
255-
MemoryRecords records = TestUtils.singletonRecords("test".getBytes(), 10L);
260+
Supplier<MemoryRecords> records = () -> TestUtils.singletonRecords("test".getBytes(), 10L);
256261
LogConfig config = new LogTestUtils.LogConfigBuilder()
257-
.withSegmentBytes(records.sizeInBytes() * 15)
262+
.withSegmentBytes(records.get().sizeInBytes() * 15)
258263
.withRetentionMs(10000L)
259264
.build();
260265
log = createLog(logDir, config);
261266

262267
for (int i = 0; i < 15; i++) {
263-
log.appendAsLeader(records, 0);
268+
log.appendAsLeader(records.get(), 0);
264269
}
265270

266271
log.updateHighWatermark(log.logEndOffset());
@@ -426,15 +431,16 @@ public void testLogDeletionAfterClose() throws IOException {
426431
.build();
427432
log = createLog(logDir, logConfig);
428433

434+
// append some messages to create some segments
429435
log.appendAsLeader(records.get(), 0);
430436

431-
assertEquals(1, log.numberOfSegments());
432-
assertEquals(1, epochCache(log).epochEntries().size());
437+
assertEquals(1, log.numberOfSegments(), "The deleted segments should be gone.");
438+
assertEquals(1, epochCache(log).epochEntries().size(), "Epoch entries should have gone.");
433439

434440
log.close();
435441
log.delete();
436442
assertEquals(0, log.numberOfSegments());
437-
assertEquals(0, epochCache(log).epochEntries().size());
443+
assertEquals(0, epochCache(log).epochEntries().size(), "Epoch entries should have gone.");
438444
}
439445

440446
@Test
@@ -459,6 +465,7 @@ public void testDeleteOldSegments() throws IOException {
459465
assertEquals(numSegments, log.numberOfSegments());
460466
assertEquals(0L, log.logStartOffset());
461467

468+
// only segments with offset before the current high watermark are eligible for deletion
462469
for (long hw = 25; hw <= 30; hw++) {
463470
log.updateHighWatermark(hw);
464471
log.deleteOldSegments();
@@ -531,41 +538,54 @@ public void shouldDeleteStartOffsetBreachedSegmentsWhenPolicyDoesNotIncludeDelet
531538

532539
@Test
533540
public void testFirstUnstableOffsetNoTransactionalData() throws IOException {
534-
Supplier<MemoryRecords> records = () -> MemoryRecords.withRecords(Compression.NONE,
541+
LogConfig logConfig = new LogTestUtils.LogConfigBuilder()
542+
.withSegmentBytes(1024 * 1024 * 5)
543+
.build();
544+
log = createLog(logDir, logConfig);
545+
546+
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
535547
new SimpleRecord("foo".getBytes()),
536548
new SimpleRecord("bar".getBytes()),
537549
new SimpleRecord("baz".getBytes()));
538550

539-
log.appendAsLeader(records.get(), 0);
551+
log.appendAsLeader(records, 0);
540552
assertEquals(Optional.empty(), log.firstUnstableOffset());
541553
}
542554

543555
@Test
544556
public void testFirstUnstableOffsetWithTransactionalData() throws IOException {
557+
LogConfig logConfig = new LogTestUtils.LogConfigBuilder()
558+
.withSegmentBytes(1024 * 1024 * 5).build();
559+
log = createLog(logDir, logConfig);
560+
545561
long pid = 137L;
546562
short epoch = 5;
547-
PrimitiveRef.IntRef seq = PrimitiveRef.ofInt(0);
563+
int seq = 0;
548564

549-
Supplier<MemoryRecords> records = () -> MemoryRecords.withTransactionalRecords(
550-
Compression.NONE, pid, epoch, seq.value,
565+
// add some transactional records
566+
MemoryRecords records = MemoryRecords.withTransactionalRecords(
567+
Compression.NONE, pid, epoch, seq,
551568
new SimpleRecord("foo".getBytes()),
552569
new SimpleRecord("bar".getBytes()),
553570
new SimpleRecord("baz".getBytes()));
554571

555-
LogAppendInfo firstAppendInfo = log.appendAsLeader(records.get(), 0);
572+
LogAppendInfo firstAppendInfo = log.appendAsLeader(records, 0);
556573
assertEquals(Optional.of(firstAppendInfo.firstOffset()), log.firstUnstableOffset());
557574

558-
seq.value += 3;
559-
log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq.value,
575+
// add more transactional records
576+
seq += 3;
577+
log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
560578
new SimpleRecord("blah".getBytes())), 0);
561-
562579
assertEquals(Optional.of(firstAppendInfo.firstOffset()), log.firstUnstableOffset());
563580

581+
// now transaction is committed
564582
LogAppendInfo commitAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, mockTime.milliseconds());
565583

584+
// first unstable offset is not updated until the high watermark is advanced
566585
assertEquals(Optional.of(firstAppendInfo.firstOffset()), log.firstUnstableOffset());
567586
log.updateHighWatermark(commitAppendInfo.lastOffset() + 1);
568587

588+
// now there should be no first unstable offset
569589
assertEquals(Optional.empty(), log.firstUnstableOffset());
570590
}
571591

0 commit comments

Comments
 (0)