kamalcph commented on code in PR #13837:
URL: https://github.com/apache/kafka/pull/13837#discussion_r1251094603


##########
storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentFileset.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.remote.storage;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.record.FileRecords;
+import org.apache.kafka.common.record.Record;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static java.lang.String.format;
+import static java.util.Arrays.stream;
+import static java.util.Collections.unmodifiableMap;
+import static java.util.Objects.requireNonNull;
+import static java.util.function.Function.identity;
+import static java.util.regex.Pattern.compile;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.LEADER_EPOCH_CHECKPOINT;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.OFFSET_INDEX;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.PRODUCER_SNAPSHOT;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.SEGMENT;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.TIME_INDEX;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.TRANSACTION_INDEX;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.getFileType;
+import static 
org.apache.kafka.server.log.remote.storage.RemoteTopicPartitionDirectory.openTopicPartitionDirectory;
+import static org.slf4j.LoggerFactory.getLogger;
+
+/**
+ * Represents the set of files offloaded to the local tiered storage for a 
single log segment.
+ * A {@link RemoteLogSegmentFileset} corresponds to the leaves of the file 
system structure of
+ * the local tiered storage:
+ *
+ * <code>
+ * / storage-directory / uuidBase64-partition-topic / 
oAtiIQ95REujbuzNd_lkLQ-segment
+ *                                                  . 
oAtiIQ95REujbuzNd_lkLQ-offset_index
+ *                                                  . 
oAtiIQ95REujbuzNd_lkLQ-time_index
+ * </code>
+ */
+public final class RemoteLogSegmentFileset {
+
+    /**
+     * The format of a file which belongs to the fileset, i.e. a file which is 
assigned to a log segment in
+     * Kafka's log directory.
+     *
+     * The name of each of the files under the scope of a log segment (the log 
file, its indexes, etc.)
+     * follows the structure UUID-FileType.
+     */
+    private static final String UUID_LEGAL_CHARS = "[a-zA-Z0-9_-]{22}";
+    private static final Pattern FILENAME_FORMAT = compile("(" + 
UUID_LEGAL_CHARS + ")-([a-z_]+)");
+    private static final int GROUP_UUID = 1;
+    private static final int GROUP_FILE_TYPE = 2;
+
+    /**
+     * Characterises the type of a file in the local tiered storage copied 
from Apache Kafka's standard storage.
+     */
+    public enum RemoteLogSegmentFileType {
+        SEGMENT(false),
+        OFFSET_INDEX(false),
+        TIME_INDEX(false),
+        TRANSACTION_INDEX(true),
+        LEADER_EPOCH_CHECKPOINT(false),
+        PRODUCER_SNAPSHOT(true);

Review Comment:
   Good catch! Updated the patch to make `producerSnapshot` file non-optional. 
We can discuss further separately whether to make it optional for a smoother 
upgrade from older server versions (<2.8).



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