TaiJuWu commented on code in PR #21223: URL: https://github.com/apache/kafka/pull/21223#discussion_r2748896372
########## tools/src/main/java/org/apache/kafka/tools/DumpLogSegments.java: ########## @@ -0,0 +1,946 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.kafka.tools; + +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.message.ConsumerProtocolAssignment; +import org.apache.kafka.common.message.ConsumerProtocolAssignmentJsonConverter; +import org.apache.kafka.common.message.ConsumerProtocolSubscription; +import org.apache.kafka.common.message.ConsumerProtocolSubscriptionJsonConverter; +import org.apache.kafka.common.message.KRaftVersionRecord; +import org.apache.kafka.common.message.KRaftVersionRecordJsonConverter; +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.LeaderChangeMessageJsonConverter; +import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotFooterRecordJsonConverter; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecordJsonConverter; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.message.VotersRecordJsonConverter; +import org.apache.kafka.common.metadata.MetadataJsonConverters; +import org.apache.kafka.common.metadata.MetadataRecordType; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.record.AbstractLegacyRecordBatch; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.FileLogInputStream; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecordSerde; +import org.apache.kafka.coordinator.common.runtime.Deserializer; +import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde; +import org.apache.kafka.coordinator.share.ShareCoordinatorRecordSerde; +import org.apache.kafka.coordinator.transaction.TransactionCoordinatorRecordSerde; +import org.apache.kafka.metadata.MetadataRecordSerde; +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.util.CommandDefaultOptions; +import org.apache.kafka.server.util.CommandLineUtils; +import org.apache.kafka.snapshot.SnapshotPath; +import org.apache.kafka.snapshot.Snapshots; +import org.apache.kafka.storage.internals.log.AbortedTxn; +import org.apache.kafka.storage.internals.log.BatchMetadata; +import org.apache.kafka.storage.internals.log.CorruptSnapshotException; +import org.apache.kafka.storage.internals.log.LogFileUtils; +import org.apache.kafka.storage.internals.log.OffsetIndex; +import org.apache.kafka.storage.internals.log.OffsetPosition; +import org.apache.kafka.storage.internals.log.ProducerStateEntry; +import org.apache.kafka.storage.internals.log.ProducerStateManager; +import org.apache.kafka.storage.internals.log.TimeIndex; +import org.apache.kafka.storage.internals.log.TimestampOffset; +import org.apache.kafka.storage.internals.log.TransactionIndex; +import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.tools.api.Decoder; +import org.apache.kafka.tools.api.StringDecoder; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.IntNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import joptsimple.OptionSpec; + +public class DumpLogSegments { + // Visible for testing + static final String RECORD_INDENT = "|"; + + public static void main(String[] args) throws IOException { + DumpLogSegmentsOptions opts = new DumpLogSegmentsOptions(args); + CommandLineUtils.maybePrintHelpOrVersion( + opts, + "This tool helps to parse a log file and dump its contents to the console, useful for debugging a seemingly corrupt log segment." + ); + opts.checkArgs(); + + Map<String, Map<Long, Long>> misMatchesForIndexFilesMap = new HashMap<>(); + TimeIndexDumpErrors timeIndexDumpErrors = new TimeIndexDumpErrors(); + Map<String, List<Pair<Long, Long>>> nonConsecutivePairsForLogFilesMap = new HashMap<>(); Review Comment: I don't think there are colliding key so change to Map. ########## tools/src/main/java/org/apache/kafka/tools/DumpLogSegments.java: ########## @@ -0,0 +1,946 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.kafka.tools; + +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.message.ConsumerProtocolAssignment; +import org.apache.kafka.common.message.ConsumerProtocolAssignmentJsonConverter; +import org.apache.kafka.common.message.ConsumerProtocolSubscription; +import org.apache.kafka.common.message.ConsumerProtocolSubscriptionJsonConverter; +import org.apache.kafka.common.message.KRaftVersionRecord; +import org.apache.kafka.common.message.KRaftVersionRecordJsonConverter; +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.LeaderChangeMessageJsonConverter; +import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotFooterRecordJsonConverter; +import org.apache.kafka.common.message.SnapshotHeaderRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecordJsonConverter; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.message.VotersRecordJsonConverter; +import org.apache.kafka.common.metadata.MetadataJsonConverters; +import org.apache.kafka.common.metadata.MetadataRecordType; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.record.AbstractLegacyRecordBatch; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.FileLogInputStream; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecordSerde; +import org.apache.kafka.coordinator.common.runtime.Deserializer; +import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde; +import org.apache.kafka.coordinator.share.ShareCoordinatorRecordSerde; +import org.apache.kafka.coordinator.transaction.TransactionCoordinatorRecordSerde; +import org.apache.kafka.metadata.MetadataRecordSerde; +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.util.CommandDefaultOptions; +import org.apache.kafka.server.util.CommandLineUtils; +import org.apache.kafka.snapshot.SnapshotPath; +import org.apache.kafka.snapshot.Snapshots; +import org.apache.kafka.storage.internals.log.AbortedTxn; +import org.apache.kafka.storage.internals.log.BatchMetadata; +import org.apache.kafka.storage.internals.log.CorruptSnapshotException; +import org.apache.kafka.storage.internals.log.LogFileUtils; +import org.apache.kafka.storage.internals.log.OffsetIndex; +import org.apache.kafka.storage.internals.log.OffsetPosition; +import org.apache.kafka.storage.internals.log.ProducerStateEntry; +import org.apache.kafka.storage.internals.log.ProducerStateManager; +import org.apache.kafka.storage.internals.log.TimeIndex; +import org.apache.kafka.storage.internals.log.TimestampOffset; +import org.apache.kafka.storage.internals.log.TransactionIndex; +import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.tools.api.Decoder; +import org.apache.kafka.tools.api.StringDecoder; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.IntNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import joptsimple.OptionSpec; + +public class DumpLogSegments { + // Visible for testing + static final String RECORD_INDENT = "|"; + + public static void main(String[] args) throws IOException { + DumpLogSegmentsOptions opts = new DumpLogSegmentsOptions(args); + CommandLineUtils.maybePrintHelpOrVersion( + opts, + "This tool helps to parse a log file and dump its contents to the console, useful for debugging a seemingly corrupt log segment." + ); + opts.checkArgs(); + + Map<String, Map<Long, Long>> misMatchesForIndexFilesMap = new HashMap<>(); + TimeIndexDumpErrors timeIndexDumpErrors = new TimeIndexDumpErrors(); + Map<String, List<Pair<Long, Long>>> nonConsecutivePairsForLogFilesMap = new HashMap<>(); + + for (String arg : opts.files()) { + File file = new File(arg); + System.out.println("Dumping " + file); + + String filename = file.getName(); + String suffix = filename.substring(filename.lastIndexOf(".")); + + switch (suffix) { + case UnifiedLog.LOG_FILE_SUFFIX, Snapshots.SUFFIX -> + dumpLog(file, opts.shouldPrintDataLog(), nonConsecutivePairsForLogFilesMap, + opts.isDeepIteration(), opts.messageParser(), opts.skipRecordMetadata(), opts.maxBytes()); + case UnifiedLog.INDEX_FILE_SUFFIX -> dumpIndex(file, opts.indexSanityOnly(), opts.verifyOnly(), + misMatchesForIndexFilesMap, opts.maxMessageSize()); + case UnifiedLog.TIME_INDEX_FILE_SUFFIX -> + dumpTimeIndex(file, opts.indexSanityOnly(), opts.verifyOnly(), timeIndexDumpErrors); + case LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX -> dumpProducerIdSnapshot(file); + case UnifiedLog.TXN_INDEX_FILE_SUFFIX -> dumpTxnIndex(file); + default -> System.err.println("Ignoring unknown file " + file); + } + } + + misMatchesForIndexFilesMap.forEach((fileName, mismatchesByOffset) -> { + System.err.println("Mismatches in :" + fileName); + mismatchesByOffset.forEach((indexOffset, logOffset) -> + System.err.println(" Index offset: " + indexOffset + ", log offset: " + logOffset)); + }); + + timeIndexDumpErrors.printErrors(); + + nonConsecutivePairsForLogFilesMap.forEach((fileName, listOfNonConsecutivePairs) -> { + System.err.println("Non-consecutive offsets in " + fileName); + listOfNonConsecutivePairs.forEach(pair -> + System.err.println(" " + pair.first + " is followed by " + pair.second)); + }); + } + + private static void dumpTxnIndex(File file) throws IOException { + try (TransactionIndex index = new TransactionIndex(UnifiedLog.offsetFromFile(file), file)) { + for (AbortedTxn abortedTxn : index.allAbortedTxns()) { + System.out.println("version: " + abortedTxn.version() + + " producerId: " + abortedTxn.producerId() + + " firstOffset: " + abortedTxn.firstOffset() + + " lastOffset: " + abortedTxn.lastOffset() + + " lastStableOffset: " + abortedTxn.lastStableOffset()); + } + } + } + + private static void dumpProducerIdSnapshot(File file) throws IOException { + try { + List<ProducerStateEntry> entries = ProducerStateManager.readSnapshot(file); + for (ProducerStateEntry entry : entries) { + System.out.print("producerId: " + entry.producerId() + + " producerEpoch: " + entry.producerEpoch() + + " coordinatorEpoch: " + entry.coordinatorEpoch() + + " currentTxnFirstOffset: " + entry.currentTxnFirstOffset() + + " lastTimestamp: " + entry.lastTimestamp()); + + if (!entry.batchMetadata().isEmpty()) { + BatchMetadata metadata = entry.batchMetadata().iterator().next(); + System.out.print(" firstSequence: " + metadata.firstSeq() + + " lastSequence: " + metadata.lastSeq() + + " lastOffset: " + metadata.lastOffset() + + " offsetDelta: " + metadata.offsetDelta() + + " timestamp: " + metadata.timestamp()); + } + System.out.println(); + } + } catch (CorruptSnapshotException e) { + System.err.println(e.getMessage()); + } + } + + // Visible for testing + static void dumpIndex(File file, + boolean indexSanityOnly, + boolean verifyOnly, + Map<String, Map<Long, Long>> misMatchesForIndexFilesMap, + int maxMessageSize) throws IOException { + long startOffset = Long.parseLong(file.getName().split("\\.")[0]); + File logFile = new File(file.getAbsoluteFile().getParent(), + file.getName().split("\\.")[0] + UnifiedLog.LOG_FILE_SUFFIX); + + try (FileRecords fileRecords = FileRecords.open(logFile, false); + OffsetIndex index = new OffsetIndex(file, startOffset, -1, false)) { + + if (index.entries() == 0) { + System.out.println(file + " is empty."); + return; + } + + // Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. + if (indexSanityOnly) { + index.sanityCheck(); + System.out.println(file + " passed sanity check."); + return; + } + + for (int i = 0; i < index.entries(); i++) { + OffsetPosition entry = index.entry(i); + + // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one + if (entry.offset() == index.baseOffset() && i > 0) { + return; + } + + FileRecords slice = fileRecords.slice(entry.position(), maxMessageSize); + long firstBatchLastOffset = slice.batches().iterator().next().lastOffset(); + if (firstBatchLastOffset != entry.offset()) { + Map<Long, Long> mismatchesByOffset = misMatchesForIndexFilesMap + .computeIfAbsent(file.getAbsolutePath(), k -> new TreeMap<>(java.util.Collections.reverseOrder())); + mismatchesByOffset.put(entry.offset(), firstBatchLastOffset); + } + if (!verifyOnly) { + System.out.println("offset: " + entry.offset() + " position: " + entry.position()); + } + } + } + } + + // Visible for testing + static void dumpTimeIndex(File file, + boolean indexSanityOnly, + boolean verifyOnly, + TimeIndexDumpErrors timeIndexDumpErrors) throws IOException { + long startOffset = Long.parseLong(file.getName().split("\\.")[0]); + File logFile = new File(file.getAbsoluteFile().getParent(), + file.getName().split("\\.")[0] + UnifiedLog.LOG_FILE_SUFFIX); + File indexFile = new File(file.getAbsoluteFile().getParent(), + file.getName().split("\\.")[0] + UnifiedLog.INDEX_FILE_SUFFIX); + + FileRecords fileRecords = null; + OffsetIndex index = null; + TimeIndex timeIndex = null; + try { + fileRecords = FileRecords.open(logFile, false); + index = new OffsetIndex(indexFile, startOffset, -1, false); + timeIndex = new TimeIndex(file, startOffset, -1, false); + + // Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. + if (indexSanityOnly) { + timeIndex.sanityCheck(); + System.out.println(file + " passed sanity check."); + return; + } + + long prevTimestamp = RecordBatch.NO_TIMESTAMP; + for (int i = 0; i < timeIndex.entries(); i++) { + TimestampOffset entry = timeIndex.entry(i); + + // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one + if (entry.offset() == timeIndex.baseOffset() && i > 0) { + return; + } + + OffsetPosition offsetPosition = index.lookup(entry.offset()); + FileRecords partialFileRecords = fileRecords.slice(offsetPosition.position(), Integer.MAX_VALUE); + List<FileLogInputStream.FileChannelRecordBatch> batches = new ArrayList<>(); + partialFileRecords.batches().forEach(batches::add); + + long maxTimestamp = RecordBatch.NO_TIMESTAMP; + // We first find the message by offset then check if the timestamp is correct. + Optional<FileLogInputStream.FileChannelRecordBatch> matchingBatch = batches.stream() + .filter(batch -> batch.lastOffset() >= entry.offset()) + .findFirst(); + + if (matchingBatch.isEmpty()) { + timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset(), -1L); + } else if (matchingBatch.get().lastOffset() != entry.offset()) { + timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset(), + matchingBatch.get().lastOffset()); + } else { + RecordBatch batch = matchingBatch.get(); + for (Record record : batch) { + maxTimestamp = Math.max(maxTimestamp, record.timestamp()); + } + + if (maxTimestamp != entry.timestamp()) { + timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp(), maxTimestamp); + } + + if (prevTimestamp >= entry.timestamp()) { + timeIndexDumpErrors.recordOutOfOrderIndexTimestamp(file, entry.timestamp(), prevTimestamp); + } + } + + if (!verifyOnly) { + System.out.println("timestamp: " + entry.timestamp() + " offset: " + entry.offset()); + } + prevTimestamp = entry.timestamp(); + } + } finally { + if (fileRecords != null) { + fileRecords.closeHandlers(); + } + if (index != null) { + index.closeHandler(); + } + if (timeIndex != null) { + timeIndex.closeHandler(); + } + } + } + + interface MessageParser<K, V> { + ParseResult<K, V> parse(Record record); + } + + record ParseResult<K, V>(Optional<K> key, Optional<V> value) { } + + static class DecoderMessageParser<K, V> implements MessageParser<K, V> { + private final Decoder<K> keyDecoder; + private final Decoder<V> valueDecoder; + + public DecoderMessageParser(Decoder<K> keyDecoder, Decoder<V> valueDecoder) { + this.keyDecoder = keyDecoder; + this.valueDecoder = valueDecoder; + } + + @Override + public ParseResult<K, V> parse(Record record) { + Optional<K> key = record.hasKey() + ? Optional.of(keyDecoder.fromBytes(Utils.readBytes(record.key()))) + : Optional.empty(); + + Optional<V> value; + if (!record.hasValue()) { + value = Optional.empty(); + } else { + value = Optional.of(valueDecoder.fromBytes(Utils.readBytes(record.value()))); + } + + return new ParseResult<>(key, value); + } + } + + /* print out the contents of the log */ + private static void dumpLog(File file, + boolean printContents, + Map<String, List<Pair<Long, Long>>> nonConsecutivePairsForLogFilesMap, + boolean isDeepIteration, + MessageParser<?, ?> parser, + boolean skipRecordMetadata, + int maxBytes) throws IOException { + if (file.getName().endsWith(UnifiedLog.LOG_FILE_SUFFIX)) { + long startOffset = Long.parseLong(file.getName().split("\\.")[0]); + System.out.println("Log starting offset: " + startOffset); + } else if (file.getName().endsWith(Snapshots.SUFFIX)) { + if (file.getName().equals(BootstrapDirectory.BINARY_BOOTSTRAP_FILENAME)) { + System.out.println("KRaft bootstrap snapshot"); + } else { + Optional<SnapshotPath> pathOpt = Snapshots.parse(file.toPath()); + System.out.println("Snapshot end offset: " + pathOpt.get().snapshotId().offset() + + ", epoch: " + pathOpt.get().snapshotId().epoch()); + } + } + + FileRecords fileRecords = null; + try { + fileRecords = FileRecords.open(file, false).slice(0, maxBytes); + long validBytes = 0L; + AtomicLong lastOffset = new AtomicLong(-1L); + + for (FileLogInputStream.FileChannelRecordBatch batch : fileRecords.batches()) { + printBatchLevel(batch, validBytes); + if (isDeepIteration) { + dumpBatchRecords(batch, lastOffset, file, nonConsecutivePairsForLogFilesMap, + skipRecordMetadata, printContents, parser); + } + validBytes += batch.sizeInBytes(); + } + + printTrailingBytes(fileRecords, validBytes, maxBytes, file); + } finally { + if (fileRecords != null) { + fileRecords.closeHandlers(); + } + } + } + + private static void dumpBatchRecords(FileLogInputStream.FileChannelRecordBatch batch, + AtomicLong lastOffset, + File file, + Map<String, List<Pair<Long, Long>>> nonConsecutivePairsForLogFilesMap, + boolean skipRecordMetadata, + boolean printContents, + MessageParser<?, ?> parser) { + for (Record record : batch) { + long previousOffset = lastOffset.get(); + if (record.offset() != previousOffset + 1) { + List<Pair<Long, Long>> nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap + .computeIfAbsent(file.getAbsolutePath(), k -> new ArrayList<>()); + nonConsecutivePairsSeq.add(0, new Pair<>(previousOffset, record.offset())); + } + lastOffset.set(record.offset()); + + if (!skipRecordMetadata) { + printRecordMetadata(batch, record); + } + + if (printContents && !batch.isControlBatch()) { + String prefix = skipRecordMetadata ? RECORD_INDENT + " " : " "; + printRecordContents(parser, record, prefix); + } + System.out.println(); + } + } + + private static void printRecordMetadata(FileLogInputStream.FileChannelRecordBatch batch, + Record record) { + System.out.print(RECORD_INDENT + " " + "offset: " + record.offset() + + " " + batch.timestampType() + ": " + record.timestamp() + + " keySize: " + record.keySize() + " valueSize: " + record.valueSize()); + + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2) { + System.out.print(" sequence: " + record.sequence() + + " headerKeys: " + java.util.Arrays.stream(record.headers()) + .map(Header::key) + .collect(Collectors.joining(",", "[", "]"))); + } + + if (record instanceof AbstractLegacyRecordBatch r) { + System.out.print(" isValid: " + r.isValid() + " crc: " + r.checksum() + "}"); Review Comment: Fix it, I think there is the same issue for old version. ``` record match { case r: AbstractLegacyRecordBatch => print(s" isValid: ${r.isValid} crc: ${r.checksum}}") case _ => ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
