hachikuji commented on a change in pull request #9512:
URL: https://github.com/apache/kafka/pull/9512#discussion_r534607264



##########
File path: raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.snapshot;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.stream.IntStream;
+import org.apache.kafka.common.record.BufferSupplier.GrowableBufferSupplier;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.record.SimpleRecord;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.raft.OffsetAndEpoch;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public final class FileRawSnapshotTest {
+    @Test
+    public void testWritingSnapshot() throws IOException {
+        Path tempDir = TestUtils.tempDirectory().toPath();
+        OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3);
+        int bufferSize = 256;
+        int batches = 10;
+        int expectedSize = 0;
+
+        try (FileRawSnapshotWriter snapshot = 
FileRawSnapshotWriter.create(tempDir, offsetAndEpoch)) {
+            assertEquals(0, snapshot.sizeInBytes());
+
+            MemoryRecords records = buildRecords(new ByteBuffer[] 
{ByteBuffer.wrap(randomBytes(bufferSize))});

Review comment:
       nit: can just do `buildRecords(ByteBuffer.wrap(randomBytes(bufferSize)))`

##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.snapshot;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.StandardOpenOption;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.raft.OffsetAndEpoch;
+
+public final class FileRawSnapshotWriter implements RawSnapshotWriter {
+    private final Path tempSnapshotPath;
+    private final FileChannel channel;
+    private final OffsetAndEpoch snapshotId;
+    private boolean frozen = false;
+
+    private FileRawSnapshotWriter(
+        Path tempSnapshotPath,
+        FileChannel channel,
+        OffsetAndEpoch snapshotId
+    ) {
+        this.tempSnapshotPath = tempSnapshotPath;
+        this.channel = channel;
+        this.snapshotId = snapshotId;
+    }
+
+    @Override
+    public OffsetAndEpoch snapshotId() {
+        return snapshotId;
+    }
+
+    @Override
+    public long sizeInBytes() throws IOException {
+        return channel.size();
+    }
+
+    @Override
+    public void append(ByteBuffer buffer) throws IOException {
+        if (frozen) {
+            throw new IllegalStateException(
+                String.format("Append not supported. Snapshot is already 
frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath)
+            );
+        }
+
+        Utils.writeFully(channel, buffer);
+    }
+
+    @Override
+    public boolean isFrozen() {
+        return frozen;
+    }
+
+    @Override
+    public void freeze() throws IOException {
+        channel.close();
+        frozen = true;
+
+        // Set readonly and ignore the result
+        if (!tempSnapshotPath.toFile().setReadOnly()) {
+            throw new IOException(String.format("Unable to set file (%s) as 
read-only", tempSnapshotPath));
+        }
+
+        Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId);
+        Files.move(tempSnapshotPath, destination, 
StandardCopyOption.ATOMIC_MOVE);

Review comment:
       It might be useful to review the history behind 
`Utils.atomicMoveWithFallback`. It's not clear to me why this case is different 
from some of the other situations that it is used.

##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java
##########
@@ -74,18 +74,19 @@ public void freeze() throws IOException {
         frozen = true;
 
         // Set readonly and ignore the result
-        if (!path.toFile().setReadOnly()) {
-            throw new IOException(String.format("Unable to set file %s as 
read-only", path));
+        if (!tempSnapshotPath.toFile().setReadOnly()) {
+            throw new IOException(String.format("Unable to set file (%s) as 
read-only", tempSnapshotPath));
         }
 
-        Path destination = Snapshots.moveRename(path, snapshotId);
-        Files.move(path, destination, StandardCopyOption.ATOMIC_MOVE);
+        Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId);
+        Files.move(tempSnapshotPath, destination, 
StandardCopyOption.ATOMIC_MOVE);
     }
 
     @Override
     public void close() throws IOException {
         channel.close();
-        Files.deleteIfExists(path);
+        // This is a noop if freeze was called before calling close
+        Files.deleteIfExists(tempSnapshotPath);

Review comment:
       Would it be useful to call this in a `finally`?

##########
File path: 
raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.snapshot;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Iterator;
+import 
org.apache.kafka.common.record.FileLogInputStream.FileChannelRecordBatch;
+import org.apache.kafka.common.record.FileRecords;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.raft.OffsetAndEpoch;
+
+public final class FileRawSnapshotReader implements RawSnapshotReader {
+    private final FileRecords fileRecords;
+    private final OffsetAndEpoch snapshotId;
+
+    private FileRawSnapshotReader(FileRecords fileRecords, OffsetAndEpoch 
snapshotId) {
+        this.fileRecords = fileRecords;
+        this.snapshotId = snapshotId;
+    }
+
+    @Override
+    public OffsetAndEpoch snapshotId() {
+        return snapshotId;
+    }
+
+    @Override
+    public long sizeInBytes() {
+        return fileRecords.sizeInBytes();
+    }
+
+    @Override
+    public Iterator<RecordBatch> iterator() {
+        return new Iterator<RecordBatch>() {

Review comment:
       In that case, I would probably just do the cast to avoid the useless 
wrapper. Maybe we could create a helper like this in `Utils`:
   
   ```java
       @SuppressWarnings("unchecked")
       private <S, T extends S> Iterator<S> covariantCast(Iterator<T> iterator) 
{
           return (Iterator<S>) iterator;
       }
   ```




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

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


Reply via email to