cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1297521425


##########
metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.controller;
+
+import org.apache.kafka.common.metadata.AbortTransactionRecord;
+import org.apache.kafka.common.metadata.BeginTransactionRecord;
+import org.apache.kafka.common.metadata.EndTransactionRecord;
+import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
+import org.apache.kafka.metadata.migration.ZkMigrationState;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ActivationRecordsGenerator {
+
+    static ControllerResult<Void> recordsForEmptyLog(
+        Logger log,
+        long transactionStartOffset,
+        boolean zkMigrationEnabled,
+        BootstrapMetadata bootstrapMetadata,
+        MetadataVersion metadataVersion
+    ) {
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        if (transactionStartOffset != -1L) {
+            // In-flight bootstrap transaction
+            if (!metadataVersion.isMetadataTransactionSupported()) {
+                throw new RuntimeException("Detected partial bootstrap records 
transaction at " +
+                    transactionStartOffset + ", but the metadata.version " + 
metadataVersion +
+                    " does not support transactions. Cannot continue.");
+            } else {
+                log.warn("Detected partial bootstrap records transaction at " 
+ transactionStartOffset +
+                    " during controller activation. Aborting this transaction 
and re-committing bootstrap records");
+                records.add(new ApiMessageAndVersion(
+                    new AbortTransactionRecord().setReason("Controller 
failover"), (short) 0));
+                records.add(new ApiMessageAndVersion(
+                    new BeginTransactionRecord().setName("Bootstrap records"), 
(short) 0));
+            }
+        } else if (metadataVersion.isMetadataTransactionSupported()) {

Review Comment:
   seems like we could get rid of the "else" here and only have one place where 
we add the BeginTransactionRecord ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to