chia7712 commented on code in PR #19607: URL: https://github.com/apache/kafka/pull/19607#discussion_r2119201116
########## server/src/test/java/org/apache/kafka/server/LogManagerIntegrationTest.java: ########## @@ -57,6 +59,68 @@ public LogManagerIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } + @ClusterTest(types = {Type.KRAFT}) + public void testIOExceptionOnLogSegmentCloseResultsInRecovery() throws IOException, InterruptedException, ExecutionException { + try (Admin admin = cluster.admin()) { + admin.createTopics(List.of(new NewTopic("foo", 1, (short) 1))).all().get(); + } + cluster.waitForTopic("foo", 1); + + // Produce some data into the topic + Map<String, Object> producerConfigs = Map.of( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers(), + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName(), + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName() + ); + + try (Producer<String, String> producer = new KafkaProducer<>(producerConfigs)) { + producer.send(new ProducerRecord<>("foo", 0, null, "bar")).get(); + producer.flush(); + } + + File timeIndexFile = cluster.brokers().get(0).logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .activeSegment() + .timeIndexFile(); + + // Set read only so that we throw an IOException on shutdown + assertTrue(timeIndexFile.exists()); + assertTrue(timeIndexFile.setReadOnly()); + + cluster.brokers().get(0).shutdown(); + + assertEquals(1, cluster.brokers().get(0).config().logDirs().size()); Review Comment: Could you please add a variable for `cluster.brokers().get(0)`? ```scala var broker = cluster.brokers().get(0); broker.shutdown(); assertEquals(1, broker.config().logDirs().size()); String logDir = broker.config().logDirs().get(0); ``` ########## storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java: ########## @@ -751,10 +751,7 @@ public Optional<FileRecords.TimestampAndOffset> findOffsetByTimestamp(long times public void close() throws IOException { if (maxTimestampAndOffsetSoFar != TimestampOffset.UNKNOWN) Utils.swallow(LOGGER, Level.WARN, "maybeAppend", () -> timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar(), true)); - Utils.closeQuietly(lazyOffsetIndex, "offsetIndex", LOGGER); - Utils.closeQuietly(lazyTimeIndex, "timeIndex", LOGGER); - Utils.closeQuietly(log, "log", LOGGER); - Utils.closeQuietly(txnIndex, "txnIndex", LOGGER); + Utils.closeAll(lazyOffsetIndex, lazyTimeIndex, log, txnIndex); Review Comment: https://github.com/apache/kafka/blob/4eac6adf91a1516f8862dc5b40ca1dfe46135695/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java#L108 ########## storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java: ########## @@ -751,10 +751,7 @@ public Optional<FileRecords.TimestampAndOffset> findOffsetByTimestamp(long times public void close() throws IOException { if (maxTimestampAndOffsetSoFar != TimestampOffset.UNKNOWN) Utils.swallow(LOGGER, Level.WARN, "maybeAppend", () -> timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar(), true)); - Utils.closeQuietly(lazyOffsetIndex, "offsetIndex", LOGGER); - Utils.closeQuietly(lazyTimeIndex, "timeIndex", LOGGER); - Utils.closeQuietly(log, "log", LOGGER); - Utils.closeQuietly(txnIndex, "txnIndex", LOGGER); + Utils.closeAll(lazyOffsetIndex, lazyTimeIndex, log, txnIndex); Review Comment: If `LogSegment#close` now throws an exception, then `LogSegments#close` might break without closing all segments, right? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org