Copilot commented on code in PR #6571:
URL: https://github.com/apache/ignite-3/pull/6571#discussion_r2334074612


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFile.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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 {
+    /**
+     * Special value that, when stored in {@link #bufferPosition}, means that 
the file is closed.
+     */
+    private static final int CLOSED_POS_MARKER = -1;
+
+    private final MappedByteBuffer buffer;
+
+    private final AtomicInteger bufferPosition = new AtomicInteger();
+
+    private final AtomicInteger numWriters = new AtomicInteger();
+
+    SegmentFile(Path path, long fileSize, long position) throws IOException {
+        if (fileSize < 0) {
+            throw new IllegalArgumentException("File size is negative: " + 
fileSize);
+        }
+
+        // FIXME: remove this limitation and replace the check with 
MAX_UNSIGNED_INT,
+        //  see https://issues.apache.org/jira/browse/IGNITE-26406
+        if (fileSize > Integer.MAX_VALUE) {
+            throw new IllegalArgumentException("File size is too big: " + 
fileSize);
+        }
+
+        try (RandomAccessFile file = openFile(path, fileSize, position)) {
+            //noinspection ChannelOpenedButNotSafelyClosed
+            buffer = file.getChannel().map(MapMode.READ_WRITE, position, 
fileSize - position);
+        }
+    }
+
+    private static RandomAccessFile openFile(Path path, long fileSize, long 
position) throws IOException {
+        if (position != 0) {
+            // Segment file already exists and has some data in it.
+            return new RandomAccessFile(path.toFile(), "rw");
+        }
+
+        try {
+            Files.createFile(path);
+        } catch (FileAlreadyExistsException ignored) {
+            // No-op.
+        }
+
+        var file = new RandomAccessFile(path.toFile(), "rw");
+
+        file.setLength(fileSize);
+
+        return file;
+    }
+
+    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) {
+        close(bytesToWrite);
+    }
+
+    @Override
+    public void close() {
+        close(null);
+    }
+
+    private void close(byte @Nullable [] bytesToWrite) {
+        int pos = bufferPosition.getAndSet(CLOSED_POS_MARKER);
+
+        if (pos == CLOSED_POS_MARKER) {
+            return;
+        }
+
+        while (numWriters.get() > 0) {
+            Thread.onSpinWait();

Review Comment:
   This busy-wait loop could consume significant CPU resources when waiting for 
writers to finish. Consider adding a brief Thread.sleep() or using a more 
efficient waiting mechanism like CountDownLatch to reduce CPU usage during the 
wait.
   ```suggestion
               try {
                   Thread.sleep(1);
               } catch (InterruptedException e) {
                   Thread.currentThread().interrupt();
                   break;
               }
   ```



##########
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.ArrayUtils.concat;
+import static org.apache.ignite.internal.util.ByteUtils.bytesToInt;
+import static org.apache.ignite.internal.util.ByteUtils.intToBytes;
+import static org.apache.ignite.internal.util.IgniteUtils.closeAllManually;
+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.InputStream;
+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.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 SegmentFile file;
+
+    @BeforeEach
+    void setUp() throws IOException {
+        path = workDir.resolve(FILE_NAME);
+
+        Files.createFile(path);
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        closeAllManually(file);
+    }
+
+    /**
+     * Tests the happy-case append scenario.
+     */
+    @Test
+    void testReserve() throws IOException {
+        createSegmentFile(300);
+
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        int dataLength = 100;
+
+        byte[] bytes1 = randomBytes(random, dataLength);
+        byte[] bytes2 = randomBytes(random, dataLength);
+
+        assertTrue(writeToSegmentFile(bytes1));
+        assertTrue(writeToSegmentFile(bytes2));
+
+        assertThat(readFileContent(bytes1.length + bytes2.length), 
is(concat(bytes1, bytes2)));
+    }
+
+    /**
+     * Tests a situation when file gets overflown with consecutive append 
calls.
+     */
+    @Test
+    void testReserveIterativeOverflow() throws IOException {
+        int fileSize = 100;
+
+        createSegmentFile(fileSize);
+
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        byte[] bytes1 = randomBytes(random, fileSize - 1);
+        byte[] bytes2 = randomBytes(random, 1);
+
+        assertTrue(writeToSegmentFile(bytes1));
+        assertFalse(writeToSegmentFile(new byte[2]));
+        assertTrue(writeToSegmentFile(bytes2));
+
+        assertThat(readFileContent(bytes1.length + bytes2.length), 
is(concat(bytes1, bytes2)));
+    }
+
+    /**
+     * Tests a situation when file gets overflown with a single big append 
call.
+     */
+    @Test
+    void testReserveSingleBatchOverflow() throws IOException {
+        int fileSize = 100;
+
+        createSegmentFile(fileSize);
+
+        assertFalse(writeToSegmentFile(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;
+
+        createSegmentFile(fileSize);
+
+        byte[] existingContent = intToBytes(239);
+
+        writeToSegmentFile(existingContent);
+
+        file.close();
+
+        openSegmentFile(Integer.BYTES);
+
+        var bytes = randomBytes(ThreadLocalRandom.current(), fileSize - 
Integer.BYTES);
+
+        assertTrue(writeToSegmentFile(bytes));
+
+        byte[] expectedBytes = ByteBuffer.allocate(bytes.length + 
Integer.BYTES)
+                .put(existingContent)
+                .put(bytes)
+                .array();
+
+        assertThat(readFileContent(expectedBytes.length), is(expectedBytes));
+    }
+
+    /**
+     * Tests that append requests return {@code false} after the file is 
closed.
+     */
+    @Test
+    void testClose() throws IOException {
+        createSegmentFile(100);
+
+        file.close();
+
+        assertFalse(writeToSegmentFile(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 {
+        createSegmentFile(100);
+
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+
+        byte[] bytes = new byte[50];
+
+        byte[] bytesForRollover = randomBytes(random, 50);
+
+        assertTrue(writeToSegmentFile(bytes));
+
+        file.closeForRollover(bytesForRollover);
+
+        assertFalse(writeToSegmentFile(new byte[1]));
+
+        assertThat(readFileContent(bytes.length + bytesForRollover.length), 
is(concat(bytes, 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;
+
+        createSegmentFile(fileSize);
+
+        var bytes = new byte[fileSize];
+
+        assertTrue(writeToSegmentFile(bytes));
+
+        file.closeForRollover(new byte[] {1, 2, 3});
+
+        assertThat(readFileContent(bytes.length), is(bytes));
+    }
+
+    /**
+     * Tests a multi-threaded happy-case append scenario. We expect that bytes 
do not get intertwined.
+     */
+    @Test
+    void testMultiThreadedReserve(@InjectExecutorService(threadCount = 10) 
ExecutorService executor) throws IOException {
+        int maxEntrySize = 100;
+
+        int fileSize = 10_000;
+
+        int numElements = fileSize / maxEntrySize;
+
+        List<byte[]> data = generateData(numElements, maxEntrySize);
+
+        createSegmentFile(fileSize);
+
+        CompletableFuture<?>[] tasks = data.stream()
+                .map(bytes -> runAsync(() -> 
assertTrue(writeToSegmentFile(bytes)), executor))
+                .toArray(CompletableFuture[]::new);
+
+        assertThat(allOf(tasks), willCompleteSuccessfully());
+
+        assertThat(readDataFromFile(numElements), 
containsInAnyOrder(data.toArray()));
+    }
+
+    /**
+     * Tests a multi-threaded append scenario when some bytes get rejected due 
to an overflow. We expect that only the
+     * successfully written bytes get written to the file.
+     */
+    @Test
+    void 
testMultiThreadedReserveWithOverflow(@InjectExecutorService(threadCount = 10) 
ExecutorService executor) throws IOException {
+        int maxEntrySize = 100;
+
+        int fileSize = 10_000;
+
+        int numElements = fileSize;
+
+        List<byte[]> data = generateData(numElements, maxEntrySize);
+
+        createSegmentFile(fileSize);
+
+        @SuppressWarnings("unchecked")
+        CompletableFuture<Boolean>[] tasks = data.stream()
+                .map(bytes -> supplyAsync(() -> writeToSegmentFile(bytes), 
executor))
+                .toArray(CompletableFuture[]::new);
+
+        assertThat(allOf(tasks), willCompleteSuccessfully());
+
+        List<byte[]> successfullyWritten = new ArrayList<>();
+        List<byte[]> notWritten = new ArrayList<>();
+
+        for (int i = 0; i < tasks.length; i++) {
+            if (tasks[i].join()) {
+                successfullyWritten.add(data.get(i));
+            } else {
+                notWritten.add(data.get(i));
+            }
+        }
+
+        assertThat(notWritten, is(not(empty())));
+
+        assertThat(readDataFromFile(successfullyWritten.size()), 
containsInAnyOrder(successfullyWritten.toArray()));
+    }
+
+    /**
+     * Tests a scenario when a file gets closed in the middle of concurrent 
append operations.
+     */
+    @Test
+    void testMultithreadedClose(@InjectExecutorService(threadCount = 10) 
ExecutorService executor) throws IOException {
+        int maxEntrySize = 100;
+
+        int fileSize = 10_000;
+
+        int numElements = fileSize / maxEntrySize;
+
+        List<byte[]> data = generateData(numElements, maxEntrySize);
+
+        createSegmentFile(fileSize);
+
+        @SuppressWarnings("unchecked")
+        CompletableFuture<Boolean>[] tasks = new 
CompletableFuture[numElements];
+
+        for (int i = 0; i < numElements; i++) {
+            byte[] bytes = data.get(i);
+
+            // Post a task to close the file somewhere in the middle.
+            if (i == numElements / 2) {
+                executor.submit(() -> file.close());
+            }
+
+            tasks[i] = supplyAsync(() -> writeToSegmentFile(bytes), executor);
+        }
+
+        assertThat(allOf(tasks), willCompleteSuccessfully());
+
+        List<byte[]> successfullyWritten = new ArrayList<>();
+        List<byte[]> notWritten = new ArrayList<>();
+
+        for (int i = 0; i < tasks.length; i++) {
+            if (tasks[i].join()) {
+                successfullyWritten.add(data.get(i));
+            } else {
+                notWritten.add(data.get(i));
+            }
+        }
+
+        assertThat(notWritten, is(not(empty())));
+
+        assertThat(readDataFromFile(successfullyWritten.size()), 
containsInAnyOrder(successfullyWritten.toArray()));
+    }
+
+    /**
+     * Tests a scenario when a file gets closed in the middle of concurrent 
append operations. We expect that rollover
+     * bytes are written at the end.
+     */
+    @Test
+    void testMultiThreadedCloseForRollover(@InjectExecutorService(threadCount 
= 10) ExecutorService executor) throws IOException {
+        int maxEntrySize = 100;
+
+        int fileSize = 10_000;
+
+        int numElements = fileSize / maxEntrySize;
+
+        List<byte[]> data = generateData(numElements, maxEntrySize);
+
+        byte[] bytesForRollover = {1, 2, 3};
+
+        createSegmentFile(fileSize);
+
+        @SuppressWarnings("unchecked")
+        CompletableFuture<Boolean>[] tasks = new 
CompletableFuture[numElements];
+
+        for (int i = 0; i < numElements; i++) {
+            byte[] bytes = data.get(i);
+
+            // Post a task to close the file somewhere in the middle.
+            if (i == numElements / 2) {
+                executor.submit(() -> file.closeForRollover(bytesForRollover));
+            }
+
+            tasks[i] = supplyAsync(() -> writeToSegmentFile(bytes), executor);
+        }
+
+        assertThat(allOf(tasks), willCompleteSuccessfully());
+
+        List<byte[]> successfullyWritten = new ArrayList<>();
+        List<byte[]> notWritten = new ArrayList<>();
+
+        for (int i = 0; i < tasks.length; i++) {
+            if (tasks[i].join()) {
+                successfullyWritten.add(data.get(i));
+            } else {
+                notWritten.add(data.get(i));
+            }
+        }
+
+        assertThat(notWritten, is(not(empty())));
+
+        assertThat(readDataFromFile(successfullyWritten.size()), 
containsInAnyOrder(successfullyWritten.toArray()));
+
+        int offset = successfullyWritten.stream().mapToInt(bytes -> 
bytes.length).sum();
+
+        assertThat(readFileContent(offset, bytesForRollover.length), 
is(bytesForRollover));
+    }
+
+    private void createSegmentFile(int size) throws IOException {
+        file = new SegmentFile(path, size, 0);
+    }
+
+    private void openSegmentFile(int position) throws IOException {
+        file = new SegmentFile(path, Files.size(path), position);
+    }
+
+    private boolean writeToSegmentFile(byte[] bytes) {
+        try (WriteBuffer writeBuffer = file.reserve(bytes.length)) {
+            if (writeBuffer == null) {
+                return false;
+            }
+
+            writeBuffer.buffer().put(bytes);
+
+            return true;
+        }
+    }
+
+    private byte[] readFileContent(int length) throws IOException {
+        return readFileContent(0, length);
+    }
+
+    private byte[] readFileContent(long offset, int length) throws IOException 
{
+        try (InputStream is = Files.newInputStream(path)) {
+            long remainingOffset = offset;
+
+            while (remainingOffset > 0) {
+                remainingOffset -= is.skip(remainingOffset);

Review Comment:
   The InputStream.skip() method may skip fewer bytes than requested and can 
return 0, potentially causing an infinite loop. Add a check to ensure progress 
is being made or use a different approach for seeking to the offset position.
   ```suggestion
                   long skipped = is.skip(remainingOffset);
                   if (skipped == 0) {
                       // Try to read and discard a byte to make progress, or 
break if EOF
                       if (is.read() == -1) {
                           // EOF reached before offset
                           throw new IOException("Reached EOF before skipping 
to offset " + offset);
                       }
                       remainingOffset--;
                   } else {
                       remainingOffset -= skipped;
                   }
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFile.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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 {
+    /**
+     * Special value that, when stored in {@link #bufferPosition}, means that 
the file is closed.
+     */
+    private static final int CLOSED_POS_MARKER = -1;
+
+    private final MappedByteBuffer buffer;
+
+    private final AtomicInteger bufferPosition = new AtomicInteger();
+
+    private final AtomicInteger numWriters = new AtomicInteger();
+
+    SegmentFile(Path path, long fileSize, long position) throws IOException {
+        if (fileSize < 0) {
+            throw new IllegalArgumentException("File size is negative: " + 
fileSize);
+        }
+
+        // FIXME: remove this limitation and replace the check with 
MAX_UNSIGNED_INT,
+        //  see https://issues.apache.org/jira/browse/IGNITE-26406
+        if (fileSize > Integer.MAX_VALUE) {
+            throw new IllegalArgumentException("File size is too big: " + 
fileSize);
+        }
+
+        try (RandomAccessFile file = openFile(path, fileSize, position)) {
+            //noinspection ChannelOpenedButNotSafelyClosed
+            buffer = file.getChannel().map(MapMode.READ_WRITE, position, 
fileSize - position);
+        }
+    }
+
+    private static RandomAccessFile openFile(Path path, long fileSize, long 
position) throws IOException {
+        if (position != 0) {
+            // Segment file already exists and has some data in it.
+            return new RandomAccessFile(path.toFile(), "rw");
+        }
+
+        try {
+            Files.createFile(path);
+        } catch (FileAlreadyExistsException ignored) {
+            // No-op.
+        }
+
+        var file = new RandomAccessFile(path.toFile(), "rw");
+
+        file.setLength(fileSize);
+
+        return file;
+    }
+
+    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) {
+        close(bytesToWrite);
+    }
+
+    @Override
+    public void close() {
+        close(null);
+    }
+
+    private void close(byte @Nullable [] bytesToWrite) {
+        int pos = bufferPosition.getAndSet(CLOSED_POS_MARKER);
+
+        if (pos == CLOSED_POS_MARKER) {
+            return;
+        }
+
+        while (numWriters.get() > 0) {
+            Thread.onSpinWait();
+        }
+
+        if (bytesToWrite != null && pos + bytesToWrite.length <= 
buffer.capacity()) {
+            slice(pos, bytesToWrite.length).put(bytesToWrite);
+        }
+    }
+
+    /**
+     * Reserves the given amount of bytes at the end of this file.
+     *
+     * <p>If the bytes have been reserved successfully, then a {@link 
WriteBuffer} instance is returned, containing a slice of the mapped
+     * byte buffer. If there's not enough space to reserve the given amount of 
bytes, then {@code null} is returned.
+     */
+    @Nullable WriteBuffer reserve(int size) {
+        numWriters.incrementAndGet();
+
+        try {
+            ByteBuffer slice = reserveBytes(size);
+
+            if (slice == null) {
+                // Not enough free space left or the file is closed.
+                numWriters.decrementAndGet();
+
+                return null;
+            }
+
+            return new WriteBuffer(slice);
+        } catch (Exception e) {
+            numWriters.decrementAndGet();
+
+            throw e;
+        }
+    }
+
+    private @Nullable ByteBuffer reserveBytes(int size) {
+        while (true) {
+            int pos = bufferPosition.get();
+
+            if (pos == CLOSED_POS_MARKER) {
+                return null;
+            }
+
+            int nextPos = pos + size;
+
+            if (nextPos > buffer.capacity()) {
+                return null;
+            }
+
+            if (bufferPosition.compareAndSet(pos, nextPos)) {
+                return slice(pos, size);
+            }
+        }
+    }
+
+    private ByteBuffer slice(int pos, int size) {
+        return buffer.duplicate().position(pos).limit(pos + size);

Review Comment:
   The ByteBuffer slice method returns a buffer without calling slice() which 
means changes to position/limit will affect the original buffer. This should be 
`buffer.duplicate().position(pos).limit(pos + size).slice()` to create an 
independent buffer slice.
   ```suggestion
           return buffer.duplicate().position(pos).limit(pos + size).slice();
   ```



-- 
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: notifications-unsubscr...@ignite.apache.org

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

Reply via email to