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 2023/01/07 16:13:39 UTC

[GitHub] [kafka] ijuma commented on a diff in pull request #13043: KAFKA-14558: Move/Rewrite LastRecord, TxnMetadata, BatchMetadata, ProducerStateEntry, and ProducerAppendInfo to the storage module

ijuma commented on code in PR #13043:
URL: https://github.com/apache/kafka/pull/13043#discussion_r1064020080


##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerAppendInfo.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.errors.InvalidProducerEpochException;
+import org.apache.kafka.common.errors.InvalidTxnStateException;
+import org.apache.kafka.common.errors.OutOfOrderSequenceException;
+import org.apache.kafka.common.errors.TransactionCoordinatorFencedException;
+import org.apache.kafka.common.record.ControlRecordType;
+import org.apache.kafka.common.record.EndTransactionMarker;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.RecordBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+
+/**
+ * This class is used to validate the records appended by a given producer before they are written to the log.
+ * It is initialized with the producer's state after the last successful append, and transitively validates the
+ * sequence numbers and epochs of each new record. Additionally, this class accumulates transaction metadata
+ * as the incoming records are validated.
+ */
+public class ProducerAppendInfo {
+    private static final Logger log = LoggerFactory.getLogger(ProducerAppendInfo.class);
+    private final TopicPartition topicPartition;
+    private final long producerId;
+    private final ProducerStateEntry currentEntry;
+    private final AppendOrigin origin;
+
+    private final List<TxnMetadata> transactions = new ArrayList<>();
+    private final ProducerStateEntry updatedEntry;
+
+    /**
+     * Creates a new instance with the provided parameters.
+     *
+     * @param topicPartition topic partition
+     * @param producerId     The id of the producer appending to the log
+     * @param currentEntry   The current entry associated with the producer id which contains metadata for a fixed number of
+     *                       the most recent appends made by the producer. Validation of the first incoming append will
+     *                       be made against the latest append in the current entry. New appends will replace older appends
+     *                       in the current entry so that the space overhead is constant.
+     * @param origin         Indicates the origin of the append which implies the extent of validation. For example, offset
+     *                       commits, which originate from the group coordinator, do not have sequence numbers and therefore
+     *                       only producer epoch validation is done. Appends which come through replication are not validated
+     *                       (we assume the validation has already been done) and appends from clients require full validation.
+     */
+    public ProducerAppendInfo(TopicPartition topicPartition,
+                              long producerId,
+                              ProducerStateEntry currentEntry,
+                              AppendOrigin origin) {
+        this.topicPartition = topicPartition;
+        this.producerId = producerId;
+        this.currentEntry = currentEntry;
+        this.origin = origin;
+
+        updatedEntry = new ProducerStateEntry(producerId, currentEntry.producerEpoch(), currentEntry.coordinatorEpoch, currentEntry.lastTimestamp, currentEntry.currentTxnFirstOffset, Optional.empty()
+        );

Review Comment:
   Nit: it looks a bit odd to have this in its own line like this. Can we fix the formatting?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateEntry.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.record.RecordBatch;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.stream.Stream;
+
+/**
+ * The batchMetadata is ordered such that the batch with the lowest sequence is at the head of the queue while the
+ * batch with the highest sequence is at the tail of the queue. We will retain at most {@link ProducerStateEntry#NUM_BATCHES_TO_RETAIN}
+ * elements in the queue. When the queue is at capacity, we remove the first element to make space for the incoming batch.
+ */
+public class ProducerStateEntry {
+    public static final int NUM_BATCHES_TO_RETAIN = 5;
+    public final long producerId;
+    private final List<BatchMetadata> batchMetadata;
+    private short producerEpoch;
+    public int coordinatorEpoch;
+    public long lastTimestamp;
+    public OptionalLong currentTxnFirstOffset;
+
+    public ProducerStateEntry(long producerId) {
+        this(producerId, new ArrayList<>(), RecordBatch.NO_PRODUCER_EPOCH, -1, RecordBatch.NO_TIMESTAMP, OptionalLong.empty());
+    }

Review Comment:
   This is one comment that was left unaddressed.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerAppendInfo.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.errors.InvalidProducerEpochException;
+import org.apache.kafka.common.errors.InvalidTxnStateException;
+import org.apache.kafka.common.errors.OutOfOrderSequenceException;
+import org.apache.kafka.common.errors.TransactionCoordinatorFencedException;
+import org.apache.kafka.common.record.ControlRecordType;
+import org.apache.kafka.common.record.EndTransactionMarker;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.RecordBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+
+/**
+ * This class is used to validate the records appended by a given producer before they are written to the log.
+ * It is initialized with the producer's state after the last successful append, and transitively validates the
+ * sequence numbers and epochs of each new record. Additionally, this class accumulates transaction metadata
+ * as the incoming records are validated.
+ */
+public class ProducerAppendInfo {
+    private static final Logger log = LoggerFactory.getLogger(ProducerAppendInfo.class);
+    private final TopicPartition topicPartition;
+    public final long producerId;
+    private final ProducerStateEntry currentEntry;
+    private final AppendOrigin origin;
+
+    private final List<TxnMetadata> transactions = new ArrayList<>();
+    private final ProducerStateEntry updatedEntry;
+
+    /**
+     * @param topicPartition topic partition
+     * @param producerId     The id of the producer appending to the log
+     * @param currentEntry   The current entry associated with the producer id which contains metadata for a fixed number of
+     *                       the most recent appends made by the producer. Validation of the first incoming append will
+     *                       be made against the latest append in the current entry. New appends will replace older appends
+     *                       in the current entry so that the space overhead is constant.
+     * @param origin         Indicates the origin of the append which implies the extent of validation. For example, offset
+     *                       commits, which originate from the group coordinator, do not have sequence numbers and therefore
+     *                       only producer epoch validation is done. Appends which come through replication are not validated
+     *                       (we assume the validation has already been done) and appends from clients require full validation.
+     */
+    public ProducerAppendInfo(TopicPartition topicPartition,
+                              long producerId,
+                              ProducerStateEntry currentEntry,
+                              AppendOrigin origin) {
+        this.topicPartition = topicPartition;
+        this.producerId = producerId;
+        this.currentEntry = currentEntry;
+        this.origin = origin;
+
+        updatedEntry = new ProducerStateEntry(producerId, currentEntry.producerEpoch(),
+                currentEntry.coordinatorEpoch,
+                currentEntry.lastTimestamp,
+                currentEntry.currentTxnFirstOffset);
+    }
+
+    private void maybeValidateDataBatch(short producerEpoch, int firstSeq, long offset) {
+        checkProducerEpoch(producerEpoch, offset);
+        if (origin == AppendOrigin.CLIENT) {
+            checkSequence(producerEpoch, firstSeq, offset);
+        }
+    }
+
+    private void checkProducerEpoch(short producerEpoch, long offset) {
+        if (producerEpoch < updatedEntry.producerEpoch()) {
+            String message = String.format("Epoch of producer %d at offset %d in %s is %d, " +
+                    "which is smaller than the last seen epoch %d", producerId, offset, topicPartition, producerEpoch, updatedEntry.producerEpoch());
+
+            if (origin == AppendOrigin.REPLICATION) {
+                log.warn(message);
+            } else {
+                // Starting from 2.7, we replaced ProducerFenced error with InvalidProducerEpoch in the
+                // producer send response callback to differentiate from the former fatal exception,
+                // letting client abort the ongoing transaction and retry.
+                throw new InvalidProducerEpochException(message);
+            }
+        }
+    }
+
+    private void checkSequence(short producerEpoch, int appendFirstSeq, long offset) {
+        if (producerEpoch != updatedEntry.producerEpoch()) {
+            if (appendFirstSeq != 0) {
+                if (updatedEntry.producerEpoch() != RecordBatch.NO_PRODUCER_EPOCH) {
+                    throw new OutOfOrderSequenceException("Invalid sequence number for new epoch of producer " + producerId +
+                            "at offset " + offset + " in partition " + topicPartition + ": " + producerEpoch + " (request epoch), "
+                            + appendFirstSeq + " (seq. number), " + updatedEntry.producerEpoch() + " (current producer epoch)");
+                }
+            }
+        } else {
+            int currentLastSeq;
+            if (!updatedEntry.isEmpty())
+                currentLastSeq = updatedEntry.lastSeq();
+            else if (producerEpoch == currentEntry.producerEpoch())
+                currentLastSeq = currentEntry.lastSeq();
+            else
+                currentLastSeq = RecordBatch.NO_SEQUENCE;
+
+            // If there is no current producer epoch (possibly because all producer records have been deleted due to
+            // retention or the DeleteRecords API) accept writes with any sequence number
+            if (!(currentEntry.producerEpoch() == RecordBatch.NO_PRODUCER_EPOCH || inSequence(currentLastSeq, appendFirstSeq))) {
+                throw new OutOfOrderSequenceException("Out of order sequence number for producer " + producerId + " at " +
+                        "offset " + offset + " in partition " + topicPartition + ": " + appendFirstSeq +
+                        " (incoming seq. number), " + currentLastSeq + " (current end sequence number)");
+            }
+        }
+    }
+
+    private boolean inSequence(int lastSeq, int nextSeq) {
+        return nextSeq == lastSeq + 1L || (nextSeq == 0 && lastSeq == Integer.MAX_VALUE);
+    }
+
+    public Optional<CompletedTxn> append(RecordBatch batch, Optional<LogOffsetMetadata> firstOffsetMetadataOpt) {
+        if (batch.isControlBatch()) {
+            Iterator<Record> recordIterator = batch.iterator();
+            if (recordIterator.hasNext()) {
+                Record record = recordIterator.next();
+                EndTransactionMarker endTxnMarker = EndTransactionMarker.deserialize(record);
+                return appendEndTxnMarker(endTxnMarker, batch.producerEpoch(), batch.baseOffset(), record.timestamp());
+            } else {
+                // An empty control batch means the entire transaction has been cleaned from the log, so no need to append
+                return Optional.empty();
+            }
+        } else {
+            LogOffsetMetadata firstOffsetMetadata = firstOffsetMetadataOpt.orElse(new LogOffsetMetadata(batch.baseOffset()));
+            appendDataBatch(batch.producerEpoch(), batch.baseSequence(), batch.lastSequence(), batch.maxTimestamp(),
+                    firstOffsetMetadata, batch.lastOffset(), batch.isTransactional());
+            return Optional.empty();
+        }
+    }
+
+    public void appendDataBatch(short epoch,
+                                int firstSeq,
+                                int lastSeq,
+                                long lastTimestamp,
+                                LogOffsetMetadata firstOffsetMetadata,
+                                long lastOffset,
+                                boolean isTransactional) {
+        long firstOffset = firstOffsetMetadata.messageOffset;
+        maybeValidateDataBatch(epoch, firstSeq, firstOffset);
+        updatedEntry.addBatch(epoch, lastSeq, lastOffset, (int) (lastOffset - firstOffset), lastTimestamp);
+
+        OptionalLong currentTxnFirstOffset = updatedEntry.currentTxnFirstOffset;
+        if (currentTxnFirstOffset.isPresent()) {
+            if (!isTransactional)

Review Comment:
   @satishd Check the old code, it is equivalent to what I proposed here.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/ProducerStateEntry.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.record.RecordBatch;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.stream.Stream;
+
+/**
+ * The batchMetadata is ordered such that the batch with the lowest sequence is at the head of the queue while the
+ * batch with the highest sequence is at the tail of the queue. We will retain at most {@link ProducerStateEntry#NUM_BATCHES_TO_RETAIN}
+ * elements in the queue. When the queue is at capacity, we remove the first element to make space for the incoming batch.
+ */
+public class ProducerStateEntry {
+    public static final int NUM_BATCHES_TO_RETAIN = 5;
+    public final long producerId;
+    private final List<BatchMetadata> batchMetadata;
+    private short producerEpoch;
+    public int coordinatorEpoch;
+    public long lastTimestamp;
+    public OptionalLong currentTxnFirstOffset;

Review Comment:
   We can leave this for a follow-up.



-- 
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