Copilot commented on code in PR #6571: URL: https://github.com/apache/ignite-3/pull/6571#discussion_r2333439241
########## modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFile.java: ########## @@ -0,0 +1,170 @@ +/* + * 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.ignite.internal.raft.storage.segstore; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.close.ManuallyCloseable; +import org.jetbrains.annotations.Nullable; + +/** + * This class represents an append-only memory-mapped segment file. + * + * <p>This implementation is thread-safe in terms of concurrent writes. + */ +class SegmentFile implements ManuallyCloseable { + private static final long MAX_UNSIGNED_INT = 0xffffffffL; + + /** + * Special value that, when stored in {@link #bufferPosition}, means that the file is closed. + */ + private static final int CLOSED_POS_MARKER = -1; + + private final RandomAccessFile file; + + private final MappedByteBuffer buffer; + + private final AtomicInteger bufferPosition = new AtomicInteger(); + + private final AtomicInteger numWriters = new AtomicInteger(); + + SegmentFile(RandomAccessFile file, long fileSize, long position) throws IOException { + if (fileSize < 0) { + throw new IllegalArgumentException("File size is negative: " + fileSize); + } + + // FIXME: remove this limitation, see https://issues.apache.org/jira/browse/IGNITE-26406 + if (fileSize > Integer.MAX_VALUE) { + throw new IllegalArgumentException("File size is too big: " + fileSize); + } + + if (fileSize > MAX_UNSIGNED_INT) { + throw new IllegalArgumentException("File size is too big: " + fileSize); + } + + this.file = file; + + //noinspection ChannelOpenedButNotSafelyClosed + buffer = file.getChannel().map(MapMode.READ_WRITE, position, fileSize - position); + } + + class WriteBuffer implements AutoCloseable { + private final ByteBuffer slice; + + WriteBuffer(ByteBuffer slice) { + this.slice = slice; + } + + ByteBuffer buffer() { + return slice; + } + + @Override + public void close() { + numWriters.decrementAndGet(); + } + } + + /** + * Closes the file with a rollover intention. This means that before the file is closed and if the file contains enough space, then the + * given bytes will be appended to the end of the file. + * + * <p>It is guaranteed that the given bytes will be written last even in presence of concurrent writers. + */ + void closeForRollover(byte[] bytesToWrite) throws IOException { + close(bytesToWrite); + } + + @Override + public void close() throws IOException { + close(null); + } + + private void close(byte @Nullable [] bytesToWrite) throws IOException { + int pos = bufferPosition.getAndSet(CLOSED_POS_MARKER); + + if (pos == CLOSED_POS_MARKER) { + return; + } + + while (numWriters.get() > 0) { + Thread.onSpinWait(); Review Comment: This spin-wait loop could consume excessive CPU resources when waiting for writers to complete. Consider adding a small sleep or timeout mechanism to prevent indefinite spinning. ```suggestion Thread.onSpinWait(); // Sleep briefly to avoid excessive CPU usage java.util.concurrent.locks.LockSupport.parkNanos(100_000); // 0.1 ms ``` ########## modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFile.java: ########## @@ -0,0 +1,170 @@ +/* + * 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.ignite.internal.raft.storage.segstore; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.close.ManuallyCloseable; +import org.jetbrains.annotations.Nullable; + +/** + * This class represents an append-only memory-mapped segment file. + * + * <p>This implementation is thread-safe in terms of concurrent writes. + */ +class SegmentFile implements ManuallyCloseable { + private static final long MAX_UNSIGNED_INT = 0xffffffffL; + + /** + * Special value that, when stored in {@link #bufferPosition}, means that the file is closed. + */ + private static final int CLOSED_POS_MARKER = -1; + + private final RandomAccessFile file; + + private final MappedByteBuffer buffer; + + private final AtomicInteger bufferPosition = new AtomicInteger(); + + private final AtomicInteger numWriters = new AtomicInteger(); + + SegmentFile(RandomAccessFile file, long fileSize, long position) throws IOException { + if (fileSize < 0) { + throw new IllegalArgumentException("File size is negative: " + fileSize); + } + + // FIXME: remove this limitation, see https://issues.apache.org/jira/browse/IGNITE-26406 + if (fileSize > Integer.MAX_VALUE) { + throw new IllegalArgumentException("File size is too big: " + fileSize); + } + + if (fileSize > MAX_UNSIGNED_INT) { + throw new IllegalArgumentException("File size is too big: " + fileSize); + } Review Comment: The check on line 56 is redundant since `Integer.MAX_VALUE` (2,147,483,647) is smaller than `MAX_UNSIGNED_INT` (4,294,967,295). The first condition will always trigger before the second one, making line 60-62 unreachable. ```suggestion ``` ########## modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileTest.java: ########## @@ -0,0 +1,457 @@ +/* + * 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.ignite.internal.raft.storage.segstore; + +import static java.util.concurrent.CompletableFuture.allOf; +import static java.util.concurrent.CompletableFuture.runAsync; +import static java.util.concurrent.CompletableFuture.supplyAsync; +import static java.util.stream.Collectors.toList; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomBytes; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; +import static org.apache.ignite.internal.util.IgniteUtils.closeAll; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.IntStream; +import org.apache.ignite.internal.raft.storage.segstore.SegmentFile.WriteBuffer; +import org.apache.ignite.internal.testframework.ExecutorServiceExtension; +import org.apache.ignite.internal.testframework.IgniteAbstractTest; +import org.apache.ignite.internal.testframework.InjectExecutorService; +import org.apache.ignite.internal.util.ByteUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +/** + * Tests for {@link SegmentFile}. + */ +@ExtendWith(ExecutorServiceExtension.class) +class SegmentFileTest extends IgniteAbstractTest { + private static final String FILE_NAME = "test.bin"; + + private Path path; + + private RandomAccessFile file; + + @BeforeEach + void setUp() throws IOException { + path = workDir.resolve(FILE_NAME); + + Files.createFile(path); + + file = new RandomAccessFile(path.toFile(), "rw"); + } + + @AfterEach + void tearDown() throws Exception { + closeAll(file); + } + + /** + * Tests the happy-case append scenario. + */ + @Test + void testReserve() throws IOException { + var segmentFile = new SegmentFile(file, 300, 0); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + + int dataLength = 100; + + byte[] bytes1 = randomBytes(random, dataLength); + byte[] bytes2 = randomBytes(random, dataLength); + + assertTrue(writeToSegmentFile(segmentFile, bytes1)); + assertTrue(writeToSegmentFile(segmentFile, bytes2)); + + assertThat(readFully(dataLength), is(bytes1)); + assertThat(readFully(dataLength), is(bytes2)); + } + + /** + * Tests a situation when file gets overflown with consecutive append calls. + */ + @Test + void testReserveIterativeOverflow() throws IOException { + int fileSize = 100; + + var segmentFile = new SegmentFile(file, fileSize, 0); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + + byte[] bytes1 = randomBytes(random, fileSize - 1); + byte[] bytes2 = randomBytes(random, 1); + + assertTrue(writeToSegmentFile(segmentFile, bytes1)); + assertFalse(writeToSegmentFile(segmentFile, new byte[2])); + assertTrue(writeToSegmentFile(segmentFile, bytes2)); + + assertThat(readFully(bytes1.length), is(bytes1)); + assertThat(readFully(bytes2.length), is(bytes2)); + } + + /** + * Tests a situation when file gets overflown with a single big append call. + */ + @Test + void testReserveSingleBatchOverflow() throws IOException { + int fileSize = 100; + + var segmentFile = new SegmentFile(file, fileSize, 0); + + assertFalse(writeToSegmentFile(segmentFile, new byte[fileSize + 1])); + } + + /** + * Tests appends to an already existing file (e.g. appends from a predetermined position). + */ + @Test + void testReserveFromPosition() throws IOException { + int fileSize = 100; + + int existingContent = 239; + + file.writeInt(existingContent); + + file.seek(0); + + var segmentFile = new SegmentFile(file, fileSize, Integer.BYTES); + + var bytes = randomBytes(ThreadLocalRandom.current(), fileSize - Integer.BYTES); + + assertTrue(writeToSegmentFile(segmentFile, bytes)); + + byte[] expectedBytes = ByteBuffer.allocate(bytes.length + Integer.BYTES) + .putInt(existingContent) + .put(bytes) + .array(); + + assertThat(readFully(fileSize), is(expectedBytes)); + } + + /** + * Tests that append requests return {@code false} after the file is closed. + */ + @Test + void testClose() throws IOException { + var segmentFile = new SegmentFile(file, 100, 0); + + segmentFile.close(); + + assertFalse(writeToSegmentFile(segmentFile, new byte[1])); + } + + /** + * Tests that append requests return {@code false} after the file is closed and rollover bytes are written at the end. + */ + @Test + void testCloseForRollover() throws IOException { + var segmentFile = new SegmentFile(file, 100, 0); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + + byte[] bytes = new byte[50]; + + byte[] bytesForRollover = randomBytes(random, 50); + + assertTrue(writeToSegmentFile(segmentFile, bytes)); + segmentFile.closeForRollover(bytesForRollover); + + assertFalse(writeToSegmentFile(segmentFile, new byte[1])); + + // Re-open the file, because "closeForRollover" closes it. + file = new RandomAccessFile(path.toFile(), "r"); + + assertThat(readFully(bytes.length), is(bytes)); + assertThat(readFully(bytesForRollover.length), is(bytesForRollover)); + } + + /** + * Tests that rollover bytes are not written if there's no space left in the file. + */ + @Test + void testCloseForRolloverOverflow() throws IOException { + int fileSize = 100; + + var segmentFile = new SegmentFile(file, 100, 0); Review Comment: The magic number `100` should use the `fileSize` variable declared on line 206 for consistency and clarity. ```suggestion var segmentFile = new SegmentFile(file, fileSize, 0); ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
