You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/12/22 13:04:44 UTC

[GitHub] [kafka] satishd opened a new pull request, #13040: KAFKA-14480 Move ProducerStateManager to storage module.

satishd opened a new pull request, #13040:
URL: https://github.com/apache/kafka/pull/13040

   KAFKA-14480 Move ProducerStateManager to storage module.
   
   This is not yet complete as it is dependant on https://issues.apache.org/jira/browse/KAFKA-14478. 
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1424214703

   Thanks for the updates, I'll review them in a couple of days.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1072167191


##########
core/src/main/scala/kafka/log/LogLoader.scala:
##########
@@ -191,7 +192,7 @@ class LogLoader(
     // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used
     // during log recovery may have deleted some files without the LogLoader.producerStateManager instance witnessing the
     // deletion.
-    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq)
+    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq.map(x => Long.box(x)).asJava)

Review Comment:
   Good catch! Avoided `Seq` conversion by using Collection as arg. We do not need these conversions when we convert LogSegment, which is planned in few weeks. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1426964876

   Sorry for missed adding the reviewers section in the merge commit.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101480397


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,20 +680,20 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {
-    producerStateManager.activeProducers.map { case (producerId, producerIdEntry) =>
-      (producerId, producerIdEntry.lastSeq)
+    producerStateManager.activeProducers.asScala.map { case (producerId, producerIdEntry) =>
+      (producerId.toLong, producerIdEntry.lastSeq)
     }
-  }
+  }.toMap

Review Comment:
   I realized, but since we are creating a new map in this method, I don't see why an immutable Map is needed. Can we just change the result type to be `collection.Map`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102512047


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+

Review Comment:
   nit: additional empty line



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+

Review Comment:
   ditto



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+

Review Comment:
   ditto



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+

Review Comment:
   ditto



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+

Review Comment:
   ditto



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,679 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);

Review Comment:
   nit: could we make them 3 lines for each method?
   return findNextIncompleteTxn(completedTxn.producerId)
     .map(x -> x.firstOffset.messageOffset)
     .orElse(completedTxn.lastOffset + 1);
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095387492


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   We are trying to avoid making changes that could result in subtle breakage unless there is a strong reason to make the change and we have a high degree of confidence that it's safe to do so. With that in mind, what's the reason for making the change you are suggesting?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1386596236

   @ijuma It looks like the changes that were done in the conversions were accidentally dropped in my local repo while doing a few rebases in my trunk. I should have checked that before pushing to PR and pinging for review. Sorry about that. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102882346


##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   It's fine for now since the test will still fail if no value is returned for the key, but it will result in a NPE instead of a NoSuchElementException.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1426961566

   All the tests are passed, merging to trunk.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095665461


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   Apologies - I just found an answer to this question [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java#L23-L28).
   
   Quoting from `KafkaStorageException`'s Javadoc:
   
   > 1) If the server has not finished loading logs, IOException does not need to be converted to KafkaStorageException
   > 2) After the server has finished loading logs, IOException should be caught and trigger LogDirFailureChannel.maybeAddOfflineLogDir(). Then the IOException should either be swallowed and logged, or be converted and re-thrown as KafkaStorageException
   > 3) It is preferred for IOException to be caught in Log rather than in ReplicaManager or LogSegment.
   
   Skimmed through the code paths in the server codebase, and I/O exceptions thrown from here are propagated up to `UnifiedLog` and log directories added there to the offline list, following the guidance from 3).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1426949004

   @satishd the latest CI run is green.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102874502


##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   aah, forgot to update the comment. I added that check initially, but later when I checked the old code, it did not have this kind of check in the old Scala code. I plan to add that as a followup, limiting this change mostly related to the refactoring.  



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101429242


##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   Added a not null check for `producerStateEntry`. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1385391553

   Thanks @ijuma for the review comments. Addressed them inline and updated the PR with the latest commit.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102933380


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {

Review Comment:
   Right, it is used only in tests. But we should not move to `LogTestUtils` as this method should take a lock while building the results. It is better to keep these semantics and leave this method here even though it is used only in tests. This will keep the method contract clean and thread safe.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102997944


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.
+     */
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and {@link Optional#empty()} will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(LogFileUtils.DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot {} with deletion suffix because it was already deleted", snapshotFile.file().getAbsoluteFile());
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version " + version);
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: " + crc + ". Computed crc: " + computedCrc);
+
+            Object[] producerEntryFields = struct.getArray(PRODUCER_ENTRIES_FIELD);
+            List<ProducerStateEntry> entries = new ArrayList<>(producerEntryFields.length);
+            for (Object producerEntryObj : producerEntryFields) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+
+                OptionalLong currentTxnFirstOffsetVal = currentTxnFirstOffset >= 0 ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty();
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, currentTxnFirstOffsetVal, batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        Struct[] structEntries = new Struct[entries.size()];
+        int i = 0;
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq())
+                    .set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries[i++] = producerEntryStruct;
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries);
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(Path path) {
+        File file = path.toFile();
+        return file.isFile() && file.getName().endsWith(LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX);

Review Comment:
   Thanks for that. I will make the change and update the PR.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1387574662

   @ijuma It looks like the changes that were done in the conversions were accidentally dropped in my local repo while doing a few rebases and squashes with trunk and other changes. I should have checked that before pushing to PR and pinging for review, which I generally follow was missed. My sincere apologies on that.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1071365497


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2104,7 +2105,7 @@ object UnifiedLog extends Logging {
     // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state
     // from the first segment.
     if (recordVersion.value < RecordBatch.MAGIC_VALUE_V2 ||
-      (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) {
+      (producerStateManager.latestSnapshotOffset.asScala.isEmpty && reloadFromCleanShutdown)) {
       // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the

Review Comment:
   Can we call !isPresent instead of asScala.isEmpty?



##########
core/src/test/scala/unit/kafka/log/LogLoaderTest.scala:
##########
@@ -557,7 +558,7 @@ class LogLoaderTest {
       _topicId = None,
       keepPartitionMetadataFile = true)
 
-    verify(stateManager).removeStraySnapshots(any[Seq[Long]])
+    verify(stateManager).removeStraySnapshots((any[java.util.List[java.lang.Long]]))

Review Comment:
   Are the extra parenthesis around `any` needed? We have a few similar examples in this file.



##########
core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala:
##########
@@ -340,8 +342,7 @@ class ProducerStateManagerTest {
     // After reloading from the snapshot, the transaction should still be considered late
     val reloadedStateManager = new ProducerStateManager(partition, logDir, maxTransactionTimeoutMs,
       producerStateManagerConfig, time)
-    reloadedStateManager.truncateAndReload(logStartOffset = 0L,
-      logEndOffset = stateManager.mapEndOffset, currentTimeMs = time.milliseconds())
+    reloadedStateManager.truncateAndReload(0L,stateManager.mapEndOffset, time.milliseconds())

Review Comment:
   Nit: space missing after `,`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManagerConfig.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.server.log.internals;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class ProducerStateManagerConfig {
+    public static final Set<String> RECONFIGURABLE_CONFIGS = Collections.singleton("producer.id.expiration.ms");
+    private volatile int producerIdExpirationMs;
+
+    public ProducerStateManagerConfig(int producerIdExpirationMs) {
+        this.producerIdExpirationMs = producerIdExpirationMs;
+    }
+
+    public void updateProducerIdExpirationMs(int producerIdExpirationMs) {

Review Comment:
   Nit: use `set` instead of `update`?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,20 +680,20 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {
-    producerStateManager.activeProducers.map { case (producerId, producerIdEntry) =>
-      (producerId, producerIdEntry.lastSeq)
+    producerStateManager.activeProducers.asScala.map { case (producerId, producerIdEntry) =>
+      (producerId.toLong, producerIdEntry.lastSeq)
     }
-  }
+  }.toMap

Review Comment:
   Can we avoid this `toMap` copy? Same for the case a few lines below.



##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -652,7 +652,7 @@ class TransactionsTest extends IntegrationTestHarness {
       producer.commitTransaction()
 
       var producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.head._2
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.asScala.head._2

Review Comment:
   Can we use `.get(0)` instead of `asScala.head`? There's one other similar example in this file.



##########
core/src/test/scala/unit/kafka/log/LogTestUtils.scala:
##########
@@ -247,7 +246,7 @@ object LogTestUtils {
   }
 
   def listProducerSnapshotOffsets(logDir: File): Seq[Long] =
-    ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted
+    ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted.toSeq

Review Comment:
   Is the `toSeq` at the end required? I'd have thought `sorted` returns a `Seq`.



##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   One difference in behavior is that Java's `Map.get` returns `null` if no mapping is found while Scala's `Map.apply` throws an exception. There's another similar example in this file.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManagerConfig.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.server.log.internals;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class ProducerStateManagerConfig {
+    public static final Set<String> RECONFIGURABLE_CONFIGS = Collections.singleton("producer.id.expiration.ms");

Review Comment:
   We should have a static final for this config name here and reference it from `KafkaConfig`.



##########
core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala:
##########
@@ -215,11 +212,11 @@ class UnifiedLogTest {
     log.close()
 
     val reopened = createLog(logDir, logConfig)
-    assertEquals(Some(new LogOffsetMetadata(3L)), reopened.producerStateManager.firstUnstableOffset)
+    assertEquals(Optional.of(new LogOffsetMetadata(3L)), reopened.producerStateManager.firstUnstableOffset)
 
     truncateFunc(reopened)(0L)
     assertEquals(None, reopened.firstUnstableOffset)
-    assertEquals(Map.empty, reopened.producerStateManager.activeProducers)
+    assertTrue(reopened.producerStateManager.activeProducers.isEmpty)

Review Comment:
   The previous approach is better as it includes the map contents in the message.



##########
core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala:
##########
@@ -88,7 +88,7 @@ class ProducerStateManagerTest {
     val producerEpoch = 2.toShort
     appendEndTxnMarker(stateManager, producerId, producerEpoch, ControlRecordType.COMMIT, offset = 27L)
 
-    val firstEntry = stateManager.lastEntry(producerId).getOrElse(throw new RuntimeException("Expected last entry to be defined"))
+    val firstEntry = stateManager.lastEntry(producerId).orElseThrow(() => throw new RuntimeException("Expected last entry to be defined"))

Review Comment:
   You are supposed to return the exception, not throw it (when using `orElseThrow`). Please update similar places in this file.



##########
core/src/main/scala/kafka/server/DynamicBrokerConfig.scala:
##########
@@ -1031,3 +1031,20 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
     listeners.map(e => (e.listenerName, e)).toMap
 
 }
+
+class DynamicProducerStateManagerConfig(val producerStateManagerConfig: ProducerStateManagerConfig) extends BrokerReconfigurable with Logging {
+  def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {
+    if (producerStateManagerConfig.producerIdExpirationMs() != newConfig.producerIdExpirationMs) {
+      info(s"Reconfigure ${KafkaConfig.ProducerIdExpirationMsProp} from ${producerStateManagerConfig.producerIdExpirationMs()} to ${newConfig.producerIdExpirationMs}")
+      producerStateManagerConfig.updateProducerIdExpirationMs(newConfig.producerIdExpirationMs)
+    }
+  }
+
+  def validateReconfiguration(newConfig: KafkaConfig): Unit = {
+    if (newConfig.producerIdExpirationMs < 0)
+      throw new ConfigException(s"${KafkaConfig.ProducerIdExpirationMsProp} cannot be less than 0, current value is ${newConfig.producerIdExpirationMs}")

Review Comment:
   In the old code, the second parameter is `producerStateManagerConfig.producerIdExpirationMs`. I think we should actually have both the current and new value.



##########
core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala:
##########
@@ -192,7 +192,8 @@ class ProducerStateManagerTest {
 
     // should be able to append with the new epoch if we start at sequence 0
     append(stateManager, producerId, bumpedProducerEpoch, 0, 2L)
-    assertEquals(Some(0), stateManager.lastEntry(producerId).map(_.firstSeq))
+    val value: Optional[Int] = stateManager.lastEntry(producerId).map(_.firstSeq)

Review Comment:
   Do we need to extract the val here?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102976553


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.
+     */
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and {@link Optional#empty()} will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(LogFileUtils.DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot {} with deletion suffix because it was already deleted", snapshotFile.file().getAbsoluteFile());
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version " + version);
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: " + crc + ". Computed crc: " + computedCrc);
+
+            Object[] producerEntryFields = struct.getArray(PRODUCER_ENTRIES_FIELD);
+            List<ProducerStateEntry> entries = new ArrayList<>(producerEntryFields.length);
+            for (Object producerEntryObj : producerEntryFields) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+
+                OptionalLong currentTxnFirstOffsetVal = currentTxnFirstOffset >= 0 ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty();
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, currentTxnFirstOffsetVal, batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        Struct[] structEntries = new Struct[entries.size()];
+        int i = 0;
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq())
+                    .set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries[i++] = producerEntryStruct;
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries);
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(Path path) {
+        File file = path.toFile();
+        return file.isFile() && file.getName().endsWith(LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX);

Review Comment:
   How about `Files.isRegularFile(path)`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095052846


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);

Review Comment:
   `Set<Long> baseOffsets`?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManagerConfig.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.server.log.internals;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class ProducerStateManagerConfig {
+    public static final Set<String> RECONFIGURABLE_CONFIGS = Collections.singleton("producer.id.expiration.ms");
+    private volatile int producerIdExpirationMs;
+
+    public ProducerStateManagerConfig(int producerIdExpirationMs) {
+        this.producerIdExpirationMs = producerIdExpirationMs;
+    }
+
+    public void updateProducerIdExpirationMs(int producerIdExpirationMs) {
+        this.producerIdExpirationMs = producerIdExpirationMs;
+    }
+
+    public int producerIdExpirationMs() {
+        return producerIdExpirationMs;
+    }
+}

Review Comment:
   Missing CR.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   Curious - is this exception eventually going to be exposed as an `UncheckedIOException` ([KAFKA-14490](https://issues.apache.org/jira/browse/KAFKA-14490))? 
   
   Is there a reason not to invoke a log failure handler (because we want to propagate to the log layer?)?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095665461


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   Apologies - I just found an answer to this question [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java#L23-L28).
   
   Quoting from `KafkaStorageException`'s Javadoc:
   
   > 1) If the server has not finished loading logs, IOException does not need to be converted to KafkaStorageException
   > 2) After the server has finished loading logs, IOException should be caught and trigger LogDirFailureChannel.maybeAddOfflineLogDir(). Then the IOException should either be swallowed and logged, or be converted and re-thrown as KafkaStorageException
   > 3) It is preferred for IOException to be caught in Log rather than in ReplicaManager or LogSegment.
   
   Checked the code path, and I/O exceptions thrown from here are propagated up to `UnifiedLog` and log directories added there to the offline list, following the guidance from 3).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101431448


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {

Review Comment:
   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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1424157795

   Thanks @ijuma for the review. Addressed them inline and/or with the latest commits. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1072167191


##########
core/src/main/scala/kafka/log/LogLoader.scala:
##########
@@ -191,7 +192,7 @@ class LogLoader(
     // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used
     // during log recovery may have deleted some files without the LogLoader.producerStateManager instance witnessing the
     // deletion.
-    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq)
+    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq.map(x => Long.box(x)).asJava)

Review Comment:
   Good catch! Avoided `Seq` conversion by using Collection as arg. We do not need these conversions when we move/rewrite LogSegment code, which is planned in few weeks. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13040: KAFKA-14480 Move ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1364050025

   Thanks KAFKA-14478, I should have a PR soon.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102882346


##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   It's fine for now since the test will still fail if no value is returned for the key, but it will result in a NPE instead of a NoSuchKeyException.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1071361752


##########
core/src/main/scala/kafka/log/LogLoader.scala:
##########
@@ -191,7 +192,7 @@ class LogLoader(
     // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used
     // during log recovery may have deleted some files without the LogLoader.producerStateManager instance witnessing the
     // deletion.
-    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq)
+    producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq.map(x => Long.box(x)).asJava)

Review Comment:
   Seems like we do two collection copies here. Maybe we can tweak things so that only one of them is needed (and when we convert the segments code, none are needed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1387745094

   @satishd No worries, it happens to all of us. :)


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095498625


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   Thanks for the follow-up. I missed to mention the question above is beyond the scope of this PR. I am trying to understand the strategy adopted in case of I/O failures in the data path. Some I/O failures directly results in putting the log directory written on offline. Some others are propagated up the stack. Is there a design pattern applying here that we have to follow? Do all I/O failures warrant the log directory to be made offline?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101481342


##########
core/src/test/scala/unit/kafka/log/LogTestUtils.scala:
##########
@@ -247,7 +246,7 @@ object LogTestUtils {
   }
 
   def listProducerSnapshotOffsets(logDir: File): Seq[Long] =
-    ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted
+    ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted.toSeq

Review Comment:
   `Buffer` is a `Seq`, but this is probably returning `immutable.Seq`. Since it's a test, we can leave as is (I hadn't noticed it was a test in the initial review.



##########
core/src/test/scala/unit/kafka/log/LogTestUtils.scala:
##########
@@ -247,7 +246,7 @@ object LogTestUtils {
   }
 
   def listProducerSnapshotOffsets(logDir: File): Seq[Long] =
-    ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted
+    ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted.toSeq

Review Comment:
   `Buffer` is a `mutable.Seq`, but this is probably returning `immutable.Seq`. Since it's a test, we can leave as is (I hadn't noticed it was a test in the initial review.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1425587517

   Thanks @showuon for your review. Addressed them with the latest commit. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102811331


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -45,7 +45,7 @@ import org.apache.kafka.server.record.BrokerCompressionType
 import org.apache.kafka.server.util.Scheduler
 import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile
 import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
-import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, EpochEntry, FetchDataInfo, FetchIsolation, LastRecord, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogValidator, ProducerAppendInfo}
+import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, EpochEntry, FetchDataInfo, FetchIsolation, LastRecord, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig}
 
 import scala.annotation.nowarn

Review Comment:
   We should remove things like `ProducerSnapshotFileSuffix` and `producerSnapshotFile` since they're now in `LogFileUtils`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);

Review Comment:
   Unnecessary `+`  between `to update` and `for partition`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -16,7 +16,12 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import java.io.File;
+import java.text.NumberFormat;
+
 public final class LogFileUtils {
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+    public static final String DELETED_FILE_SUFFIX = ".deleted";

Review Comment:
   Worth adding the comment from `LocalLog`:
   
   `/** a file that is scheduled to be deleted */`



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -32,4 +37,31 @@ public static long offsetFromFileName(String fileName) {
         return Long.parseLong(fileName.substring(0, fileName.indexOf('.')));
     }
 
+    /**
+     * Returns a File instance with parent directory as logDir and the file name as producer snapshot file for the
+     * given offset.
+     *
+     * @param logDir parent directory for the snapshot file
+     * @param offset offset number for the snapshot file
+     * @return a File instance for producer snapshot.

Review Comment:
   This documentation is different from the `UnifiedLog` one. In particular, the details for `offset` are important (and now missing).



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.
+     */
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and {@link Optional#empty()} will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(LogFileUtils.DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot {} with deletion suffix because it was already deleted", snapshotFile.file().getAbsoluteFile());
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version " + version);
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: " + crc + ". Computed crc: " + computedCrc);

Review Comment:
   The first `+` seems unnecessary.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);

Review Comment:
   Closing bracket should be `]`, not `)`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.

Review Comment:
   Looks like a word is missing after `if any IOException`?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");

Review Comment:
   Space missing before `which`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);

Review Comment:
   This is not a copy unlike what the documentation says. If we don't actually need to copy, we should change the documentation. Looking at the code, it does seem like we do _not_ need a copy, so we probably just need to update the documentation.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.
+     */
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and {@link Optional#empty()} will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(LogFileUtils.DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot {} with deletion suffix because it was already deleted", snapshotFile.file().getAbsoluteFile());
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version " + version);
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: " + crc + ". Computed crc: " + computedCrc);
+
+            Object[] producerEntryFields = struct.getArray(PRODUCER_ENTRIES_FIELD);
+            List<ProducerStateEntry> entries = new ArrayList<>(producerEntryFields.length);
+            for (Object producerEntryObj : producerEntryFields) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+
+                OptionalLong currentTxnFirstOffsetVal = currentTxnFirstOffset >= 0 ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty();
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, currentTxnFirstOffsetVal, batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        Struct[] structEntries = new Struct[entries.size()];
+        int i = 0;
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq())
+                    .set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries[i++] = producerEntryStruct;
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries);
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(Path path) {
+        File file = path.toFile();
+        return file.isFile() && file.getName().endsWith(LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX);

Review Comment:
   I think we can implement this directly on the path without converting to File.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {
-    producerStateManager.activeProducers.map { case (producerId, producerIdEntry) =>
-      (producerId, producerIdEntry.lastSeq)
+    val result = mutable.Map[Long, Int]()
+    producerStateManager.activeProducers.forEach { case (producerId, producerIdEntry) =>
+      result.put(producerId.toLong, producerIdEntry.lastSeq)
     }
+    result.toMap
   }
 
   private[log] def lastRecordsOfActiveProducers: Map[Long, LastRecord] = lock synchronized {

Review Comment:
   We should return `scala.collection.Map` here and remove the `toMap` at the end of the implementation.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,20 +680,20 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {
-    producerStateManager.activeProducers.map { case (producerId, producerIdEntry) =>
-      (producerId, producerIdEntry.lastSeq)
+    producerStateManager.activeProducers.asScala.map { case (producerId, producerIdEntry) =>
+      (producerId.toLong, producerIdEntry.lastSeq)
     }
-  }
+  }.toMap

Review Comment:
   Resolving as I added more specific comments in the last review.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -32,4 +37,31 @@ public static long offsetFromFileName(String fileName) {
         return Long.parseLong(fileName.substring(0, fileName.indexOf('.')));
     }
 
+    /**
+     * Returns a File instance with parent directory as logDir and the file name as producer snapshot file for the
+     * given offset.
+     *
+     * @param logDir parent directory for the snapshot file
+     * @param offset offset number for the snapshot file
+     * @return a File instance for producer snapshot.
+     */
+    public static File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX);

Review Comment:
   `LogFileUtils.` is redundant since it's the same file?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {

Review Comment:
   Is it right that this is only used in tests? If so, can we move this to `LogTestUtils`? Then we don't have to worry about the `toMap` copy.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.

Review Comment:
   Nit: there should be a space before `(exlusive)`.



##########
core/src/test/scala/integration/kafka/api/TransactionsTest.scala:
##########
@@ -685,7 +685,7 @@ class TransactionsTest extends IntegrationTestHarness {
       // get here without having bumped the epoch. If bumping the epoch is possible, the producer will attempt to, so
       // check there that the epoch has actually increased
       producerStateEntry =
-        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers(producerId)
+        brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId)

Review Comment:
   I don't see a not null check for `producerStateEntry`, am I missing something?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {

Review Comment:
   Looks like this wasn't addressed, but I added a new comment related to it in the latest review, so resolving this one.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102842008


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {

Review Comment:
   Is it right that this is only used in tests? If so, can we move this to `LogTestUtils`? Then we don't have to worry about the `toMap` copy. Alternatively, just change the return type to `scala.collection.Map` and remove the unnecessary `toMap` at the end.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13040:
URL: https://github.com/apache/kafka/pull/13040#issuecomment-1426831625

   Looks like the last commit caused a number of test failures.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1073049904


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());

Review Comment:
   Nit: you can pass the class, no need to get `getName()`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");

Review Comment:
   This needs updating not to use string interpolation as it doesn't work in Java.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.

Review Comment:
   `_logDir` doesn't exist in the Java class.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");

Review Comment:
   Replace string interpolation with log interpolation.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());

Review Comment:
   Could this be a `List` instead of `Set`? Not sure we need the latter here.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";

Review Comment:
   We have `LogFileUtils` for these kinds of things.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));

Review Comment:
   The formatting of this is odd. The formatting we used in the Scala code looks more readable to me.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {

Review Comment:
   We can assign the array to a variable and then allocate `entries` with the right size.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, (currentTxnFirstOffset >= 0) ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty(), batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        List<Struct> structEntries = new ArrayList<>(entries.size());
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq()).set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries.add(producerEntryStruct);
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries.toArray());
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(File file) {
+        return file.getName().endsWith(PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    // visible for testing
+    public static List<SnapshotFile> listSnapshotFiles(File dir) throws IOException {
+        if (dir.exists() && dir.isDirectory()) {
+            try (Stream<Path> paths = Files.list(dir.toPath())) {

Review Comment:
   Worth noting that we changed the implementation to use `Files.list`. This is an improvement, but we should call out these changes in the PR description.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, (currentTxnFirstOffset >= 0) ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty(), batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        List<Struct> structEntries = new ArrayList<>(entries.size());
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq()).set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries.add(producerEntryStruct);
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries.toArray());
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(File file) {
+        return file.getName().endsWith(PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    // visible for testing
+    public static List<SnapshotFile> listSnapshotFiles(File dir) throws IOException {
+        if (dir.exists() && dir.isDirectory()) {
+            try (Stream<Path> paths = Files.list(dir.toPath())) {
+                return paths.filter(path -> path.toFile().isFile() && isSnapshotFile(path.toFile())).map(path -> new SnapshotFile(path.toFile())).collect(Collectors.toList());

Review Comment:
   It's inefficient to make multiple `toFile` calls for each file. If we want to keep this improved implementation, we should change `isSnapshotFile` to work with `Path` and use the relevant method to check that a `Path` is a file.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, (currentTxnFirstOffset >= 0) ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty(), batchMetadata));

Review Comment:
   I think the old code where `currentTxnFirstOffsetValue` was a separate variable was more readable.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {

Review Comment:
   We should add a javadoc that explains that this method is thread safe (the Scala class had an annotation indicating that).



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;

Review Comment:
   Let's group the fields based on whether they're `final`, `volatile`, mutable, etc.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, (currentTxnFirstOffset >= 0) ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty(), batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        List<Struct> structEntries = new ArrayList<>(entries.size());
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq()).set(LAST_OFFSET_FIELD, entry.lastDataOffset())

Review Comment:
   We should have a single `set` per line.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {

Review Comment:
   The documentation says a copy is returned, but the implementation does not do that.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version $version");
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: $crc. Computed crc: $computedCrc");
+
+            List<ProducerStateEntry> entries = new ArrayList<>();
+            for (Object producerEntryObj : struct.getArray(PRODUCER_ENTRIES_FIELD)) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, (currentTxnFirstOffset >= 0) ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty(), batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        List<Struct> structEntries = new ArrayList<>(entries.size());

Review Comment:
   We can avoid a copy by using Struct[] here. Since we know the size, the code will be similar, the main difference is having to keep an index that we increment inline as we add an item to the array (e.g. `structEntries[i++]  = producerEntryStruct`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;

Review Comment:
   Why do we need this? Seems like we can inline it by using `return` within the if/else.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());

Review Comment:
   The original code used `snapshot` instead of `snapshot.file()` for the log parameter.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {

Review Comment:
   The old code had a `visible for testing` comment. Is that not true?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);

Review Comment:
   Use log interpolation.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");

Review Comment:
   Replace string interpolation - doesn't work in Java.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);

Review Comment:
   `producers.keySet().removeAll()` is probably more efficient.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.

Review Comment:
   Comment must be updated not to refer to `None`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());

Review Comment:
   Shall we call this `currentEntry` like in the old code?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {

Review Comment:
   We should document that the method is thread-safe (like the old class).



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());

Review Comment:
   Also, we have a `logIdent` defined for the class logger, so we probably need a non static logger.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");

Review Comment:
   String interpolation must be fixed.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {

Review Comment:
   This and the next private method should be in `LogFileUtils`?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot ${snapshot.file.getAbsoluteFile} with deletion suffix because it was already deleted");

Review Comment:
   Use log interpolation, not string interpolation.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    private File producerSnapshotFile(File logDir, long offset) {
+        return new File(logDir, filenamePrefixFromOffset(offset) + PRODUCER_SNAPSHOT_FILE_SUFFIX);
+    }
+
+    /**
+     * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
+     * so that ls sorts the files numerically.
+     *
+     * @param offset The offset to use in the file name
+     * @return The filename
+     */
+    private String filenamePrefixFromOffset(long offset) {
+        NumberFormat nf = NumberFormat.getInstance();
+        nf.setMinimumIntegerDigits(20);
+        nf.setMaximumFractionDigits(0);
+        nf.setGroupingUsed(false);
+        return nf.format(offset);
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId).map(x -> x.firstOffset.messageOffset).orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction $completedTxn on partition $topicPartition " + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and None will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {

Review Comment:
   Did we make this public due to the callers being in different files (previously same package, but no longer)?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1095665461


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateManager.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.server.log.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+    private static final Logger log = LoggerFactory.getLogger(ProducerStateManager.class.getName());
+
+    // Remove these once UnifiedLog moves to storage module.
+    public static final String DELETED_FILE_SUFFIX = ".deleted";
+    public static final String PRODUCER_SNAPSHOT_FILE_SUFFIX = ".snapshot";
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA = new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"), new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"), new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"), new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"), new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"), new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"), new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"), new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA = new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"), new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"), new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+    private final TopicPartition topicPartition;
+    private volatile File logDir;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+
+    /**
+     * Load producer state snapshots by scanning the _logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> result;
+        if (!unreplicatedFirstOffset.isPresent()) result = undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent()) result = unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            result = undecidedFirstOffset;
+        else result = unreplicatedFirstOffset;
+
+        return result;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot.file());
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '${snapshot.file}': ${e.getMessage}");
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        Set<Long> keys = producers.entrySet().stream().filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())).map(Map.Entry::getKey).collect(Collectors.toSet());
+        for (Long key : keys) {
+            producers.remove(key);
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition $topicPartition");
+
+        log.trace("Updated producer " + appendInfo.producerId() + " state to" + appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry producerStateEntry = producers.get(appendInfo.producerId());
+        if (producerStateEntry != null) {
+            producerStateEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {

Review Comment:
   Apologies - I just found an answer to this question [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java#L23-L28).
   
   Quoting from `KafkaStorageException`'s Javadoc:
   
   > 1) If the server has not finished loading logs, IOException does not need to be converted to KafkaStorageException
   > 2) After the server has finished loading logs, IOException should be caught and trigger LogDirFailureChannel.maybeAddOfflineLogDir(). Then the IOException should either be swallowed and logged, or be converted and re-thrown as KafkaStorageException
   > 3) It is preferred for IOException to be caught in Log rather than in ReplicaManager or LogSegment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102975376


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {

Review Comment:
   Makes sense about keeping the implementation in `UnifiedLog`. I still think we should simply return `collection.Map` and avoid the copy.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,19 +680,23 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {

Review Comment:
   Makes sense about keeping the implementation in `UnifiedLog`. I still think we should simply return `collection.Map` and avoid the extra copy.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd merged pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd merged PR #13040:
URL: https://github.com/apache/kafka/pull/13040


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1102914037


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java:
##########
@@ -0,0 +1,676 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Maintains a mapping from ProducerIds to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ * <p>
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ * <p>
+ * As long as a producer id is contained in the map, the corresponding producer can continue to write data.
+ * However, producer ids can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given producer id is retained in the log provided it hasn't expired due to
+ * age. This ensures that producer ids will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+public class ProducerStateManager {
+
+    public static final long LATE_TRANSACTION_BUFFER_MS = 5 * 60 * 1000;
+
+    private static final short PRODUCER_SNAPSHOT_VERSION = 1;
+    private static final String VERSION_FIELD = "version";
+    private static final String CRC_FIELD = "crc";
+    private static final String PRODUCER_ID_FIELD = "producer_id";
+    private static final String LAST_SEQUENCE_FIELD = "last_sequence";
+    private static final String PRODUCER_EPOCH_FIELD = "epoch";
+    private static final String LAST_OFFSET_FIELD = "last_offset";
+    private static final String OFFSET_DELTA_FIELD = "offset_delta";
+    private static final String TIMESTAMP_FIELD = "timestamp";
+    private static final String PRODUCER_ENTRIES_FIELD = "producer_entries";
+    private static final String COORDINATOR_EPOCH_FIELD = "coordinator_epoch";
+    private static final String CURRENT_TXN_FIRST_OFFSET_FIELD = "current_txn_first_offset";
+
+    private static final int VERSION_OFFSET = 0;
+    private static final int CRC_OFFSET = VERSION_OFFSET + 2;
+    private static final int PRODUCER_ENTRIES_OFFSET = CRC_OFFSET + 4;
+
+    private static final Schema PRODUCER_SNAPSHOT_ENTRY_SCHEMA =
+            new Schema(new Field(PRODUCER_ID_FIELD, Type.INT64, "The producer ID"),
+                    new Field(PRODUCER_EPOCH_FIELD, Type.INT16, "Current epoch of the producer"),
+                    new Field(LAST_SEQUENCE_FIELD, Type.INT32, "Last written sequence of the producer"),
+                    new Field(LAST_OFFSET_FIELD, Type.INT64, "Last written offset of the producer"),
+                    new Field(OFFSET_DELTA_FIELD, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+                    new Field(TIMESTAMP_FIELD, Type.INT64, "Max timestamp from the last written entry"),
+                    new Field(COORDINATOR_EPOCH_FIELD, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+                    new Field(CURRENT_TXN_FIRST_OFFSET_FIELD, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"));
+    private static final Schema PID_SNAPSHOT_MAP_SCHEMA =
+            new Schema(new Field(VERSION_FIELD, Type.INT16, "Version of the snapshot file"),
+                    new Field(CRC_FIELD, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+                    new Field(PRODUCER_ENTRIES_FIELD, new ArrayOf(PRODUCER_SNAPSHOT_ENTRY_SCHEMA), "The entries in the producer table"));
+
+    private final Logger log;
+
+    private final TopicPartition topicPartition;
+    private final int maxTransactionTimeoutMs;
+    private final ProducerStateManagerConfig producerStateManagerConfig;
+    private final Time time;
+
+    private final Map<Long, ProducerStateEntry> producers = new HashMap<>();
+
+    // ongoing transactions sorted by the first offset of the transaction
+    private final TreeMap<Long, TxnMetadata> ongoingTxns = new TreeMap<>();
+
+    // completed transactions whose markers are at offsets above the high watermark
+    private final TreeMap<Long, TxnMetadata> unreplicatedTxns = new TreeMap<>();
+
+    private volatile File logDir;
+
+    // Keep track of the last timestamp from the oldest transaction. This is used
+    // to detect (approximately) when a transaction has been left hanging on a partition.
+    // We make the field volatile so that it can be safely accessed without a lock.
+    private volatile long oldestTxnLastTimestamp = -1L;
+
+    private ConcurrentSkipListMap<Long, SnapshotFile> snapshots;
+    private long lastMapOffset = 0L;
+    private long lastSnapOffset = 0L;
+
+    public ProducerStateManager(TopicPartition topicPartition, File logDir, int maxTransactionTimeoutMs, ProducerStateManagerConfig producerStateManagerConfig, Time time) throws IOException {
+        this.topicPartition = topicPartition;
+        this.logDir = logDir;
+        this.maxTransactionTimeoutMs = maxTransactionTimeoutMs;
+        this.producerStateManagerConfig = producerStateManagerConfig;
+        this.time = time;
+        log = new LogContext("[ProducerStateManager partition=" + topicPartition + ")").logger(ProducerStateManager.class);
+        snapshots = loadSnapshots();
+    }
+
+    public int maxTransactionTimeoutMs() {
+        return maxTransactionTimeoutMs;
+    }
+
+    public ProducerStateManagerConfig producerStateManagerConfig() {
+        return producerStateManagerConfig;
+    }
+
+    /**
+     * This method checks whether there is a late transaction in a thread safe manner.
+     */
+    public boolean hasLateTransaction(long currentTimeMs) {
+        long lastTimestamp = oldestTxnLastTimestamp;
+        return lastTimestamp > 0 && (currentTimeMs - lastTimestamp) > maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS;
+    }
+
+    public void truncateFullyAndReloadSnapshots() throws IOException {
+        log.info("Reloading the producer state snapshots");
+        truncateFullyAndStartAt(0L);
+        snapshots = loadSnapshots();
+    }
+
+    /**
+     * Load producer state snapshots by scanning the logDir.
+     */
+    private ConcurrentSkipListMap<Long, SnapshotFile> loadSnapshots() throws IOException {
+        ConcurrentSkipListMap<Long, SnapshotFile> offsetToSnapshots = new ConcurrentSkipListMap<>();
+        List<SnapshotFile> snapshotFiles = listSnapshotFiles(logDir);
+        for (SnapshotFile snapshotFile : snapshotFiles) {
+            offsetToSnapshots.put(snapshotFile.offset, snapshotFile);
+        }
+        return offsetToSnapshots;
+    }
+
+    /**
+     * Scans the log directory, gathering all producer state snapshot files. Snapshot files which do not have an offset
+     * corresponding to one of the provided offsets in segmentBaseOffsets will be removed, except in the case that there
+     * is a snapshot file at a higher offset than any offset in segmentBaseOffsets.
+     * <p>
+     * The goal here is to remove any snapshot files which do not have an associated segment file, but not to remove the
+     * largest stray snapshot file which was emitted during clean shutdown.
+     */
+    public void removeStraySnapshots(Collection<Long> segmentBaseOffsets) throws IOException {
+        OptionalLong maxSegmentBaseOffset = segmentBaseOffsets.isEmpty() ? OptionalLong.empty() : OptionalLong.of(segmentBaseOffsets.stream().max(Long::compare).get());
+
+        HashSet<Long> baseOffsets = new HashSet<>(segmentBaseOffsets);
+        Optional<SnapshotFile> latestStraySnapshot = Optional.empty();
+
+        ConcurrentSkipListMap<Long, SnapshotFile> snapshots = loadSnapshots();
+        for (SnapshotFile snapshot : snapshots.values()) {
+            long key = snapshot.offset;
+            if (latestStraySnapshot.isPresent()) {
+                SnapshotFile prev = latestStraySnapshot.get();
+                if (!baseOffsets.contains(key)) {
+                    // this snapshot is now the largest stray snapshot.
+                    prev.deleteIfExists();
+                    snapshots.remove(prev.offset);
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            } else {
+                if (!baseOffsets.contains(key)) {
+                    latestStraySnapshot = Optional.of(snapshot);
+                }
+            }
+        }
+
+        // Check to see if the latestStraySnapshot is larger than the largest segment base offset, if it is not,
+        // delete the largestStraySnapshot.
+        if (latestStraySnapshot.isPresent() && maxSegmentBaseOffset.isPresent()) {
+            long strayOffset = latestStraySnapshot.get().offset;
+            long maxOffset = maxSegmentBaseOffset.getAsLong();
+            if (strayOffset < maxOffset) {
+                SnapshotFile removedSnapshot = snapshots.remove(strayOffset);
+                if (removedSnapshot != null) {
+                    removedSnapshot.deleteIfExists();
+                }
+            }
+        }
+
+        this.snapshots = snapshots;
+    }
+
+    /**
+     * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+     * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+     * marker written at a higher offset than the current high watermark).
+     */
+    public Optional<LogOffsetMetadata> firstUnstableOffset() {
+        Optional<LogOffsetMetadata> unreplicatedFirstOffset = Optional.ofNullable(unreplicatedTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+        Optional<LogOffsetMetadata> undecidedFirstOffset = Optional.ofNullable(ongoingTxns.firstEntry()).map(e -> e.getValue().firstOffset);
+
+        if (!unreplicatedFirstOffset.isPresent())
+            return undecidedFirstOffset;
+        else if (!undecidedFirstOffset.isPresent())
+            return unreplicatedFirstOffset;
+        else if (undecidedFirstOffset.get().messageOffset < unreplicatedFirstOffset.get().messageOffset)
+            return undecidedFirstOffset;
+        else
+            return unreplicatedFirstOffset;
+    }
+
+    /**
+     * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+     * to advance to the next unstable offset.
+     */
+    public void onHighWatermarkUpdated(long highWatermark) {
+        removeUnreplicatedTransactions(highWatermark);
+    }
+
+    /**
+     * The first undecided offset is the earliest transactional message which has not yet been committed
+     * or aborted. Unlike [[firstUnstableOffset]], this does not reflect the state of replication (i.e.
+     * whether a completed transaction marker is beyond the high watermark).
+     */
+    public OptionalLong firstUndecidedOffset() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        return firstEntry != null ? OptionalLong.of(firstEntry.getValue().firstOffset.messageOffset) : OptionalLong.empty();
+    }
+
+    /**
+     * Returns the last offset of this map
+     */
+    public long mapEndOffset() {
+        return lastMapOffset;
+    }
+
+    /**
+     * Get a copy of the active producers
+     */
+    public Map<Long, ProducerStateEntry> activeProducers() {
+        return Collections.unmodifiableMap(producers);
+    }
+
+    public boolean isEmpty() {
+        return producers.isEmpty() && unreplicatedTxns.isEmpty();
+    }
+
+    private void loadFromSnapshot(long logStartOffset, long currentTime) throws IOException {
+        while (true) {
+            Optional<SnapshotFile> latestSnapshotFileOptional = latestSnapshotFile();
+            if (latestSnapshotFileOptional.isPresent()) {
+                SnapshotFile snapshot = latestSnapshotFileOptional.get();
+                try {
+                    log.info("Loading producer state from snapshot file '{}'", snapshot);
+                    Stream<ProducerStateEntry> loadedProducers = readSnapshot(snapshot.file()).stream().filter(producerEntry -> !isProducerExpired(currentTime, producerEntry));
+                    loadedProducers.forEach(this::loadProducerEntry);
+                    lastSnapOffset = snapshot.offset;
+                    lastMapOffset = lastSnapOffset;
+                    updateOldestTxnTimestamp();
+                    return;
+                } catch (CorruptSnapshotException e) {
+                    log.warn("Failed to load producer snapshot from '{}': {}", snapshot.file(), e.getMessage());
+                    removeAndDeleteSnapshot(snapshot.offset);
+                }
+            } else {
+                lastSnapOffset = logStartOffset;
+                lastMapOffset = logStartOffset;
+                return;
+
+            }
+        }
+    }
+
+    // Visible for testing
+    public void loadProducerEntry(ProducerStateEntry entry) {
+        long producerId = entry.producerId();
+        producers.put(producerId, entry);
+        entry.currentTxnFirstOffset().ifPresent(offset -> ongoingTxns.put(offset, new TxnMetadata(producerId, offset)));
+    }
+
+    private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry producerState) {
+        return !producerState.currentTxnFirstOffset().isPresent() && currentTimeMs - producerState.lastTimestamp() >= producerStateManagerConfig.producerIdExpirationMs();
+    }
+
+    /**
+     * Expire any producer ids which have been idle longer than the configured maximum expiration timeout.
+     */
+    public void removeExpiredProducers(long currentTimeMs) {
+        List<Long> keys = producers.entrySet().stream()
+                .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue()))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+        producers.keySet().removeAll(keys);
+    }
+
+    /**
+     * Truncate the producer id mapping to the given offset range and reload the entries from the most recent
+     * snapshot in range (if there is one). We delete snapshot files prior to the logStartOffset but do not remove
+     * producer state from the map. This means that in-memory and on-disk state can diverge, and in the case of
+     * broker failover or unclean shutdown, any in-memory state not persisted in the snapshots will be lost, which
+     * would lead to UNKNOWN_PRODUCER_ID errors. Note that the log end offset is assumed to be less than or equal
+     * to the high watermark.
+     */
+    public void truncateAndReload(long logStartOffset, long logEndOffset, long currentTimeMs) throws IOException {
+        // remove all out of range snapshots
+        for (SnapshotFile snapshot : snapshots.values()) {
+            if (snapshot.offset > logEndOffset || snapshot.offset <= logStartOffset) {
+                removeAndDeleteSnapshot(snapshot.offset);
+            }
+        }
+
+        if (logEndOffset != mapEndOffset()) {
+            producers.clear();
+            ongoingTxns.clear();
+            updateOldestTxnTimestamp();
+
+            // since we assume that the offset is less than or equal to the high watermark, it is
+            // safe to clear the unreplicated transactions
+            unreplicatedTxns.clear();
+            loadFromSnapshot(logStartOffset, currentTimeMs);
+        } else {
+            onLogStartOffsetIncremented(logStartOffset);
+        }
+    }
+
+    public ProducerAppendInfo prepareUpdate(long producerId, AppendOrigin origin) {
+        ProducerStateEntry currentEntry = lastEntry(producerId).orElse(ProducerStateEntry.empty(producerId));
+        return new ProducerAppendInfo(topicPartition, producerId, currentEntry, origin);
+    }
+
+    /**
+     * Update the mapping with the given append information
+     */
+    public void update(ProducerAppendInfo appendInfo) {
+        if (appendInfo.producerId() == RecordBatch.NO_PRODUCER_ID)
+            throw new IllegalArgumentException("Invalid producer id " + appendInfo.producerId() + " passed to update " + "for partition" + topicPartition);
+
+        log.trace("Updated producer {} state to {}", appendInfo.producerId(), appendInfo);
+        ProducerStateEntry updatedEntry = appendInfo.toEntry();
+        ProducerStateEntry currentEntry = producers.get(appendInfo.producerId());
+        if (currentEntry != null) {
+            currentEntry.update(updatedEntry);
+        } else {
+            producers.put(appendInfo.producerId(), updatedEntry);
+        }
+
+        appendInfo.startedTransactions().forEach(txn -> ongoingTxns.put(txn.firstOffset.messageOffset, txn));
+
+        updateOldestTxnTimestamp();
+    }
+
+    private void updateOldestTxnTimestamp() {
+        Map.Entry<Long, TxnMetadata> firstEntry = ongoingTxns.firstEntry();
+        if (firstEntry == null) {
+            oldestTxnLastTimestamp = -1;
+        } else {
+            TxnMetadata oldestTxnMetadata = firstEntry.getValue();
+            ProducerStateEntry entry = producers.get(oldestTxnMetadata.producerId);
+            oldestTxnLastTimestamp = entry != null ? entry.lastTimestamp() : -1L;
+        }
+    }
+
+    public void updateMapEndOffset(long lastOffset) {
+        lastMapOffset = lastOffset;
+    }
+
+    /**
+     * Get the last written entry for the given producer id.
+     */
+    public Optional<ProducerStateEntry> lastEntry(long producerId) {
+        return Optional.ofNullable(producers.get(producerId));
+    }
+
+    /**
+     * Take a snapshot at the current end offset if one does not already exist.
+     */
+    public void takeSnapshot() throws IOException {
+        // If not a new offset, then it is not worth taking another snapshot
+        if (lastMapOffset > lastSnapOffset) {
+            SnapshotFile snapshotFile = new SnapshotFile(LogFileUtils.producerSnapshotFile(logDir, lastMapOffset));
+            long start = time.hiResClockMs();
+            writeSnapshot(snapshotFile.file(), producers);
+            log.info("Wrote producer snapshot at offset {} with {} producer ids in {} ms.", lastMapOffset,
+                    producers.size(), time.hiResClockMs() - start);
+
+            snapshots.put(snapshotFile.offset, snapshotFile);
+
+            // Update the last snap offset according to the serialized map
+            lastSnapOffset = lastMapOffset;
+        }
+    }
+
+    /**
+     * Update the parentDir for this ProducerStateManager and all of the snapshot files which it manages.
+     */
+    public void updateParentDir(File parentDir) {
+        logDir = parentDir;
+        snapshots.forEach((k, v) -> v.updateParentDir(parentDir));
+    }
+
+    /**
+     * Get the last offset (exclusive) of the latest snapshot file.
+     */
+    public OptionalLong latestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = latestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Get the last offset (exclusive) of the oldest snapshot file.
+     */
+    public OptionalLong oldestSnapshotOffset() {
+        Optional<SnapshotFile> snapshotFileOptional = oldestSnapshotFile();
+        return snapshotFileOptional.map(snapshotFile -> OptionalLong.of(snapshotFile.offset)).orElseGet(OptionalLong::empty);
+    }
+
+    /**
+     * Visible for testing
+     */
+    public Optional<SnapshotFile> snapshotFileForOffset(long offset) {
+        return Optional.ofNullable(snapshots.get(offset));
+    }
+
+    /**
+     * Remove any unreplicated transactions lower than the provided logStartOffset and bring the lastMapOffset forward
+     * if necessary.
+     */
+    public void onLogStartOffsetIncremented(long logStartOffset) {
+        removeUnreplicatedTransactions(logStartOffset);
+
+        if (lastMapOffset < logStartOffset) lastMapOffset = logStartOffset;
+
+        lastSnapOffset = latestSnapshotOffset().orElse(logStartOffset);
+    }
+
+    private void removeUnreplicatedTransactions(long offset) {
+        Iterator<Map.Entry<Long, TxnMetadata>> iterator = unreplicatedTxns.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, TxnMetadata> txnEntry = iterator.next();
+            OptionalLong lastOffset = txnEntry.getValue().lastOffset;
+            if (lastOffset.isPresent() && lastOffset.getAsLong() < offset) iterator.remove();
+        }
+    }
+
+    /**
+     * Truncate the producer id mapping and remove all snapshots. This resets the state of the mapping.
+     */
+    public void truncateFullyAndStartAt(long offset) throws IOException {
+        producers.clear();
+        ongoingTxns.clear();
+        unreplicatedTxns.clear();
+        for (SnapshotFile snapshotFile : snapshots.values()) {
+            removeAndDeleteSnapshot(snapshotFile.offset);
+        }
+        lastSnapOffset = 0L;
+        lastMapOffset = offset;
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete.
+     * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the
+     * transaction index, but the completion must be done only after successfully appending to the index.
+     */
+    public long lastStableOffset(CompletedTxn completedTxn) {
+        return findNextIncompleteTxn(completedTxn.producerId)
+                .map(x -> x.firstOffset.messageOffset)
+                .orElse(completedTxn.lastOffset + 1);
+    }
+
+    private Optional<TxnMetadata> findNextIncompleteTxn(long producerId) {
+        for (TxnMetadata txnMetadata : ongoingTxns.values()) {
+            if (txnMetadata.producerId != producerId) {
+                return Optional.of(txnMetadata);
+            }
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * Mark a transaction as completed. We will still await advancement of the high watermark before
+     * advancing the first unstable offset.
+     */
+    public void completeTxn(CompletedTxn completedTxn) {
+        TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset);
+        if (txnMetadata == null)
+            throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + "which was not started");
+
+        txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset);
+        unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata);
+        updateOldestTxnTimestamp();
+    }
+
+    /**
+     * Deletes the producer snapshot files until the given offset(exclusive) in a thread safe manner.
+     *
+     * @param offset offset number
+     * @throws IOException if any IOException while deleting the files.
+     */
+    public void deleteSnapshotsBefore(long offset) throws IOException {
+        for (SnapshotFile snapshot : snapshots.subMap(0L, offset).values()) {
+            removeAndDeleteSnapshot(snapshot.offset);
+        }
+    }
+
+    private Optional<SnapshotFile> oldestSnapshotFile() {
+        return Optional.ofNullable(snapshots.firstEntry()).map(x -> x.getValue());
+    }
+
+    private Optional<SnapshotFile> latestSnapshotFile() {
+        return Optional.ofNullable(snapshots.lastEntry()).map(e -> e.getValue());
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and deletes the backing snapshot file.
+     */
+    private void removeAndDeleteSnapshot(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) snapshotFile.deleteIfExists();
+    }
+
+    /**
+     * Removes the producer state snapshot file metadata corresponding to the provided offset if it exists from this
+     * ProducerStateManager, and renames the backing snapshot file to have the Log.DeletionSuffix.
+     * <p>
+     * Note: This method is safe to use with async deletes. If a race occurs and the snapshot file
+     * is deleted without this ProducerStateManager instance knowing, the resulting exception on
+     * SnapshotFile rename will be ignored and {@link Optional#empty()} will be returned.
+     */
+    public Optional<SnapshotFile> removeAndMarkSnapshotForDeletion(long snapshotOffset) throws IOException {
+        SnapshotFile snapshotFile = snapshots.remove(snapshotOffset);
+        if (snapshotFile != null) {
+            // If the file cannot be renamed, it likely means that the file was deleted already.
+            // This can happen due to the way we construct an intermediate producer state manager
+            // during log recovery, and use it to issue deletions prior to creating the "real"
+            // producer state manager.
+            //
+            // In any case, removeAndMarkSnapshotForDeletion is intended to be used for snapshot file
+            // deletion, so ignoring the exception here just means that the intended operation was
+            // already completed.
+            try {
+                snapshotFile.renameTo(LogFileUtils.DELETED_FILE_SUFFIX);
+                return Optional.of(snapshotFile);
+            } catch (NoSuchFileException ex) {
+                log.info("Failed to rename producer state snapshot {} with deletion suffix because it was already deleted", snapshotFile.file().getAbsoluteFile());
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static List<ProducerStateEntry> readSnapshot(File file) throws IOException {
+        try {
+            byte[] buffer = Files.readAllBytes(file.toPath());
+            Struct struct = PID_SNAPSHOT_MAP_SCHEMA.read(ByteBuffer.wrap(buffer));
+
+            Short version = struct.getShort(VERSION_FIELD);
+            if (version != PRODUCER_SNAPSHOT_VERSION)
+                throw new CorruptSnapshotException("Snapshot contained an unknown file version " + version);
+
+            long crc = struct.getUnsignedInt(CRC_FIELD);
+            long computedCrc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.length - PRODUCER_ENTRIES_OFFSET);
+            if (crc != computedCrc)
+                throw new CorruptSnapshotException("Snapshot is corrupt (CRC is no longer valid). " + "Stored crc: " + crc + ". Computed crc: " + computedCrc);
+
+            Object[] producerEntryFields = struct.getArray(PRODUCER_ENTRIES_FIELD);
+            List<ProducerStateEntry> entries = new ArrayList<>(producerEntryFields.length);
+            for (Object producerEntryObj : producerEntryFields) {
+                Struct producerEntryStruct = (Struct) producerEntryObj;
+                long producerId = producerEntryStruct.getLong(PRODUCER_ID_FIELD);
+                short producerEpoch = producerEntryStruct.getShort(PRODUCER_EPOCH_FIELD);
+                int seq = producerEntryStruct.getInt(LAST_SEQUENCE_FIELD);
+                long offset = producerEntryStruct.getLong(LAST_OFFSET_FIELD);
+                long timestamp = producerEntryStruct.getLong(TIMESTAMP_FIELD);
+                int offsetDelta = producerEntryStruct.getInt(OFFSET_DELTA_FIELD);
+                int coordinatorEpoch = producerEntryStruct.getInt(COORDINATOR_EPOCH_FIELD);
+                long currentTxnFirstOffset = producerEntryStruct.getLong(CURRENT_TXN_FIRST_OFFSET_FIELD);
+
+                OptionalLong currentTxnFirstOffsetVal = currentTxnFirstOffset >= 0 ? OptionalLong.of(currentTxnFirstOffset) : OptionalLong.empty();
+                Optional<BatchMetadata> batchMetadata =
+                        (offset >= 0) ? Optional.of(new BatchMetadata(seq, offset, offsetDelta, timestamp)) : Optional.empty();
+                entries.add(new ProducerStateEntry(producerId, producerEpoch, coordinatorEpoch, timestamp, currentTxnFirstOffsetVal, batchMetadata));
+            }
+
+            return entries;
+        } catch (SchemaException e) {
+            throw new CorruptSnapshotException("Snapshot failed schema validation: " + e.getMessage());
+        }
+    }
+
+    private static void writeSnapshot(File file, Map<Long, ProducerStateEntry> entries) throws IOException {
+        Struct struct = new Struct(PID_SNAPSHOT_MAP_SCHEMA);
+        struct.set(VERSION_FIELD, PRODUCER_SNAPSHOT_VERSION);
+        struct.set(CRC_FIELD, 0L); // we'll fill this after writing the entries
+        Struct[] structEntries = new Struct[entries.size()];
+        int i = 0;
+        for (Map.Entry<Long, ProducerStateEntry> producerIdEntry : entries.entrySet()) {
+            Long producerId = producerIdEntry.getKey();
+            ProducerStateEntry entry = producerIdEntry.getValue();
+            Struct producerEntryStruct = struct.instance(PRODUCER_ENTRIES_FIELD);
+            producerEntryStruct.set(PRODUCER_ID_FIELD, producerId)
+                    .set(PRODUCER_EPOCH_FIELD, entry.producerEpoch())
+                    .set(LAST_SEQUENCE_FIELD, entry.lastSeq())
+                    .set(LAST_OFFSET_FIELD, entry.lastDataOffset())
+                    .set(OFFSET_DELTA_FIELD, entry.lastOffsetDelta())
+                    .set(TIMESTAMP_FIELD, entry.lastTimestamp())
+                    .set(COORDINATOR_EPOCH_FIELD, entry.coordinatorEpoch())
+                    .set(CURRENT_TXN_FIRST_OFFSET_FIELD, entry.currentTxnFirstOffset().orElse(-1L));
+            structEntries[i++] = producerEntryStruct;
+        }
+        struct.set(PRODUCER_ENTRIES_FIELD, structEntries);
+
+        ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf());
+        struct.writeTo(buffer);
+        buffer.flip();
+
+        // now fill in the CRC
+        long crc = Crc32C.compute(buffer, PRODUCER_ENTRIES_OFFSET, buffer.limit() - PRODUCER_ENTRIES_OFFSET);
+        ByteUtils.writeUnsignedInt(buffer, CRC_OFFSET, crc);
+
+        try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            fileChannel.write(buffer);
+            fileChannel.force(true);
+        }
+    }
+
+    private static boolean isSnapshotFile(Path path) {
+        File file = path.toFile();
+        return file.isFile() && file.getName().endsWith(LogFileUtils.PRODUCER_SNAPSHOT_FILE_SUFFIX);

Review Comment:
   I am not sure there is a way to find whether a given path instance is a `file` or not like the way we check `File.isFile()`. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101430927


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -680,20 +680,20 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized {
-    producerStateManager.activeProducers.map { case (producerId, producerIdEntry) =>
-      (producerId, producerIdEntry.lastSeq)
+    producerStateManager.activeProducers.asScala.map { case (producerId, producerIdEntry) =>
+      (producerId.toLong, producerIdEntry.lastSeq)
     }
-  }
+  }.toMap

Review Comment:
   This is done to return an immutable `Map` from here. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13040: KAFKA-14480 Move/Rewrite ProducerStateManager to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13040:
URL: https://github.com/apache/kafka/pull/13040#discussion_r1101428207


##########
core/src/test/scala/unit/kafka/log/LogTestUtils.scala:
##########
@@ -247,7 +246,7 @@ object LogTestUtils {
   }
 
   def listProducerSnapshotOffsets(logDir: File): Seq[Long] =
-    ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted
+    ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted.toSeq

Review Comment:
   `sorted` returns a `Buffer` but not a `Seq`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org