jsancio commented on a change in pull request #11529:
URL: https://github.com/apache/kafka/pull/11529#discussion_r757663204



##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java
##########
@@ -14,17 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.kafka.snapshot;
 
+import org.apache.kafka.raft.Batch;
 import org.apache.kafka.raft.OffsetAndEpoch;
+
 import java.util.Iterator;
-import org.apache.kafka.raft.Batch;
 
 /**
- * Interface of the snapshot reader
+ * A type for reading an immutable snapshot.
+ *
+ * A snapshot reader can be used to scan through all of the objects T in a 
snapshot. It
+ * is assumed that the content of the snapshot represents all of the objects T 
for the topic
+ * partition from offset 0 up to but not including the end offset in the 
snapshot id.
+ *
+ * The offsets ({@code baseOffset()} and {@code lastOffset()} stored in {@code 
Batch<T>}
+ * objects returned by this iterator are independent of the offset of the 
records in the
+ * log used to generate this batch.
+ *
+ * Use {@code lastContainedLogOffset()} and {@code lastContainedLogEpoch()} to 
query which
+ * offsets and epoch from the log are included in this snapshot. Both of these 
values are
+ * inclusive.
  */
-public interface FileSnapshotReader<T> extends AutoCloseable, 
Iterator<Batch<T>> {
+public interface RecordsSnapshotReader<T> extends AutoCloseable, 
Iterator<Batch<T>> {

Review comment:
       This interface should be named `public interface SnapshotReader<T> ...`.

##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
##########
@@ -37,14 +52,12 @@
 
     /**
      * Returns true if the snapshot has been frozen, otherwise false is 
returned.
-     * <p>

Review comment:
       Please a newline between the two Java Doc section. This comment applies 
to two other places in this file. E.g.
   ```
       /**
        * Returns true if the snapshot has been frozen, otherwise false is 
returned.
        *
        * Modification to the snapshot are not allowed once it is frozen.
        */
   ```

##########
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -366,15 +367,15 @@ void createSnapshotGenerator(long committedOffset, int 
committedEpoch, long comm
                     )
                 );
             }
-            Optional<SnapshotWriter<ApiMessageAndVersion>> writer = 
raftClient.createSnapshot(
+            Optional<RecordsSnapshotWriter<ApiMessageAndVersion>> writer = 
raftClient.createSnapshot(
                 committedOffset,
                 committedEpoch,
                 committedTimestamp
             );
             if (writer.isPresent()) {
                 generator = new SnapshotGenerator(
                     logContext,
-                    writer.get(),
+                        (SnapshotWriter<ApiMessageAndVersion>) writer.get(),

Review comment:
       Please remove this cast. With the changes I suggested in my other 
comments the type of `writer` will be `Optional<SnapshotWriter>`. 
`SnapshotGenerator` should expect the type 
`SnapshotWriter<ApiMessageAndVersion>` for the second argument to the 
constructor.

##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
##########
@@ -15,11 +15,26 @@
  * limitations under the License.
  */
 package org.apache.kafka.snapshot;
+
 import org.apache.kafka.raft.OffsetAndEpoch;
 import org.apache.kafka.common.message.SnapshotFooterRecord;
 
 import java.util.List;
-public interface FileSnapshotWriter<T> extends AutoCloseable {
+
+/**
+ * A type for writing a snapshot for a given end offset and epoch.
+ *
+ * A snapshot writer can be used to append objects until freeze is called. 
When freeze is
+ * called the snapshot is validated and marked as immutable. After freeze is 
called any
+ * append will fail with an exception.
+ *
+ * It is assumed that the content of the snapshot represents all of the 
records for the
+ * topic partition from offset 0 up to but not including the end offset in the 
snapshot
+ * id.
+ *
+ * @see org.apache.kafka.raft.KafkaRaftClient#createSnapshot(long, int, long)
+ */
+public interface RecordsSnapshotWriter<T> extends AutoCloseable {

Review comment:
       This interface should be name `public interface SnapshotWriter<T> ...`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/RaftClient.java
##########
@@ -49,23 +49,23 @@
 
         /**
          * Callback which is invoked when the Listener needs to load a 
snapshot.
-         * It is the responsibility of this implementation to invoke {@link 
FileSnapshotReader#close()}
+         * It is the responsibility of this implementation to invoke {@link 
RecordsSnapshotReader#close()}
          * after consuming the reader.
          *
          * When handling this call, the implementation must assume that all 
previous calls
          * to {@link #handleCommit} contain invalid data.
          *
          * @param reader snapshot reader instance which must be iterated and 
closed
          */
-        void handleSnapshot(FileSnapshotReader<T> reader);
+        void handleSnapshot(RecordsSnapshotReader<T> reader);

Review comment:
       The type of the `reader` argument should be `SnapshotReader<T>`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/RaftClient.java
##########
@@ -219,5 +219,5 @@ default void beginShutdown() {}
      * @throws IllegalArgumentException if the committed offset is greater 
than the high-watermark
      *         or less than the log start offset.
      */
-    Optional<FileSnapshotWriter<T>> createSnapshot(long committedOffset, int 
committedEpoch, long lastContainedLogTime);
+    Optional<RecordsSnapshotWriter<T>> createSnapshot(long committedOffset, 
int committedEpoch, long lastContainedLogTime);

Review comment:
       The returned type should be `Optional<SnapshotWriter<T>>`.

##########
File path: raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java
##########
@@ -27,22 +27,7 @@
 import org.apache.kafka.server.common.serialization.RecordSerde;
 import org.apache.kafka.raft.internals.RecordsIterator;
 
-/**
- * A type for reading an immutable snapshot.
- *
- * A snapshot reader can be used to scan through all of the objects T in a 
snapshot. It
- * is assumed that the content of the snapshot represents all of the objects T 
for the topic
- * partition from offset 0 up to but not including the end offset in the 
snapshot id.
- *
- * The offsets ({@code baseOffset()} and {@code lastOffset()} stored in {@code 
Batch<T>}
- * objects returned by this iterator are independent of the offset of the 
records in the
- * log used to generate this batch.
- *
- * Use {@code lastContainedLogOffset()} and {@code lastContainedLogEpoch()} to 
query which
- * offsets and epoch from the log are included in this snapshot. Both of these 
values are
- * inclusive.
- */
-public final class SnapshotReader<T> implements FileSnapshotReader<T> {
+public final class SnapshotReader<T> implements RecordsSnapshotReader<T> {

Review comment:
       Please change the name of this class to `RecordsSnapshotReader<T>`.

##########
File path: raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java
##########
@@ -32,20 +32,7 @@
 import java.util.List;
 import java.util.function.Supplier;
 
-/**
- * A type for writing a snapshot for a given end offset and epoch.
- *
- * A snapshot writer can be used to append objects until freeze is called. 
When freeze is
- * called the snapshot is validated and marked as immutable. After freeze is 
called any
- * append will fail with an exception.
- *
- * It is assumed that the content of the snapshot represents all of the 
records for the
- * topic partition from offset 0 up to but not including the end offset in the 
snapshot
- * id.
- *
- * @see org.apache.kafka.raft.KafkaRaftClient#createSnapshot(long, int, long)
- */
-final public class SnapshotWriter<T> implements FileSnapshotWriter<T> {
+final public class SnapshotWriter<T> implements RecordsSnapshotWriter<T> {

Review comment:
       Please change the name of this class to `RecordsSnapshotWriter<T>`.




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