You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2021/06/18 20:55:49 UTC

[GitHub] [bookkeeper] zymap opened a new pull request #2742: Abstract the FileChannel in the JournalChannel

zymap opened a new pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742


   
   ### Motivation
   
   Make the FileChannel in the JournalChannel can use different implement.
   We found we can use [pmemstore](https://github.com/4paradigm/pmemstore)
   as the JournalChannel read from. So abstract the FileChannel in the
   JournnalChannel to make us can have a different implementation.
   
   ### Changes
   
   - Add interface for supporting implement different FileChannel
   
   This PR doesn't introduce any new things in the code. So make sure it can pass the CI
   


-- 
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:
users@infra.apache.org



[GitHub] [bookkeeper] eolivelli merged pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli merged pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742


   


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-908302073


   merging


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r693790962



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {

Review comment:
       I have add a "close(BookieFileChannel)" method below. Do you mean one provider can only  related to one BookieFileChannel? Should I change it to "close()"?  @eolivelli 




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-897356866


   rerun failure checks


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-903412478


   rerun failure checks


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-902567598


   `FileChannelProvider ` do not hold any resources, maybe you mean add `close() `to BookieFileChannel? `close()` method can release `RandomAccessFile` and its `FileChannel` @eolivelli .


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-906510008


   rerun failure checks


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r690937048



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getDeclaredConstructor().newInstance();

Review comment:
       it is better to use `getConstructor()` instead of `getDeclaredConstructor()` (includes "private" methods)

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannelProvider.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * A wrapper of FileChannel.
+ */
+public class DefaultFileChannelProvider implements FileChannelProvider{
+    @Override
+    public BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException {
+        return new DefaultFileChannel(file, configuration);
+    }
+
+    static class DefaultFileChannel implements BookieFileChannel {
+        private final File file;
+        private RandomAccessFile randomAccessFile;
+        private final ServerConfiguration configuration;
+
+        DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+            this.file = file;
+            this.configuration = serverConfiguration;
+        }
+
+        @Override
+        public FileChannel getFileChannel() throws FileNotFoundException {
+            synchronized (this) {
+                if (randomAccessFile == null) {
+                    randomAccessFile = new RandomAccessFile(file, "rw");
+                }
+                return randomAccessFile.getChannel();
+            }
+        }
+
+        @Override
+        public boolean fileExists(File file) {
+            return file.exists();
+        }
+
+        @Override
+        public FileDescriptor getFD() throws IOException {
+            synchronized (this) {
+                return randomAccessFile.getFD();

Review comment:
       NPE?
   I suggest to throw IOException if the handle is null
   Or to initialise it by calling `getFileChannel`




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] hangc0276 commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-900762769


   @eolivelli @merlimat @sijie Please help review this PR, thanks.


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] zymap commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-898313753


   @eolivelli @merlimat @hangc0276 Please take a review at this PR, we just want to open an interface to allow to support different FileChannel providers so it's may have some custom implementation on that. And we already send a discussion about this in the ML. https://lists.apache.org/thread.html/r0abbe15a9b7c501b602d44d954800217cc80268c7d67dd350187a710%40%3Cdev.bookkeeper.apache.org%3E
   
   It's better to include this change into the 4.14.2 release. Thanks!
   
   /cc @Sunny-Island 


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r693792726



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return FileChannelProvider. A file channel provider loaded from providerClassName
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getConstructor().newInstance();
+            return (FileChannelProvider) obj;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * Get the BookieFileChannel with the given file and configuration.
+     *
+     * @param file
+     * @param configuration
+     * @return BookieFileChannel related to file parameter.
+     * @throws IOException
+     */
+    BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException;
+
+    void close(BookieFileChannel bookieFileChannel) throws IOException;

Review comment:
       Should I change it to "close()" and create a BookieFileChannel member opened by this provider?




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-906057217


   @eolivelli Hi, could you have a look at my comments?


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] hangc0276 commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r689210122



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getDeclaredConstructor().newInstance();

Review comment:
       Why use `getDeclaredConstructor`?

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -85,37 +86,41 @@
     // The position of the file channel's last drop position
     private long lastDropPosition = 0L;
 
+    final ServerConfiguration configuration;
+
     // Mostly used by tests
     JournalChannel(File journalDirectory, long logId) throws IOException {
-        this(journalDirectory, logId, 4 * 1024 * 1024, 65536, START_OF_FILE);
+        this(journalDirectory, logId, 4 * 1024 * 1024, 65536, START_OF_FILE, new ServerConfiguration());

Review comment:
       just define `MB = 1024 * 1024` maybe better.

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -143,27 +148,32 @@
     private JournalChannel(File journalDirectory, long logId,
                            long preAllocSize, int writeBufferSize, int journalAlignSize,
                            long position, boolean fRemoveFromPageCache,
-                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder) throws IOException {
+                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder,
+                           ServerConfiguration configuration) throws IOException {
         this.journalAlignSize = journalAlignSize;
         this.zeros = ByteBuffer.allocate(journalAlignSize);
         this.preAllocSize = preAllocSize - preAllocSize % journalAlignSize;
         this.fRemoveFromPageCache = fRemoveFromPageCache;
+        this.configuration = configuration;
+
         File fn = new File(journalDirectory, Long.toHexString(logId) + ".txn");
+        FileChannelProvider provider;
+        provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());

Review comment:
       ```Java
   FileChannelProvider provider;
   provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());
   ```
   to 
   `FileChannelProvider provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());`

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return

Review comment:
       Please add docs for the return type. The same as other interfaces.




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] zymap commented on pull request #2742: [WIP] Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-865456314


   Thanks, @dlg99 and @eolivelli for your review, I will open a discussion on the ML.
   


-- 
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:
users@infra.apache.org



[GitHub] [bookkeeper] hangc0276 commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-900762769


   @eolivelli @merlimat @sijie Please help review this PR, thanks.


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r697961557



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannel.java
##########
@@ -0,0 +1,75 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+class DefaultFileChannel implements BookieFileChannel {
+    private final File file;
+    private RandomAccessFile randomAccessFile;
+    private final ServerConfiguration configuration;
+
+    DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+        this.file = file;
+        this.configuration = serverConfiguration;
+    }
+
+    @Override
+    public FileChannel getFileChannel() throws FileNotFoundException {
+        synchronized (this) {
+            if (randomAccessFile == null) {
+                randomAccessFile = new RandomAccessFile(file, "rw");
+            }
+            return randomAccessFile.getChannel();
+        }
+    }
+
+    @Override
+    public boolean fileExists(File file) {
+        return file.exists();
+    }
+
+    @Override
+    public FileDescriptor getFD() throws IOException {
+        synchronized (this) {
+            if (randomAccessFile == null) {
+                throw new IOException("randomAccessFile is null, please initialize it by calling getFileChannel");
+            }
+            return randomAccessFile.getFD();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        synchronized (this) {

Review comment:
       Ok.




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] zymap commented on pull request #2742: [WIP] Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-865456314


   Thanks, @dlg99 and @eolivelli for your review, I will open a discussion on the ML.
   


-- 
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:
users@infra.apache.org



[GitHub] [bookkeeper] dlg99 commented on pull request #2742: [WIP] Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
dlg99 commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-864501198


   @zymap change broke most of the test, I see stuff like 
   ```
   Error:    Run 1: LedgerStorageCheckpointTest.testCheckPointForEntryLoggerWithMultipleActiveEntryLogs:613->lambda$testCheckPointForEntryLoggerWithMultipleActiveEntryLogs$2:614->lambda$null$1:616 NullPointer
   ```
   
   pmemstore is not apache licensed, its addition should be discussed with the community along with the expected outcome, Reading from the Journal is not a bottleneck normally. Can you start a thread in a dev maillist to initiate the discussion?
   


-- 
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:
users@infra.apache.org



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r696649727



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return FileChannelProvider. A file channel provider loaded from providerClassName
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getConstructor().newInstance();
+            return (FileChannelProvider) obj;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * Get the BookieFileChannel with the given file and configuration.
+     *
+     * @param file
+     * @param configuration
+     * @return BookieFileChannel related to file parameter.
+     * @throws IOException
+     */
+    BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException;
+
+    void close(BookieFileChannel bookieFileChannel) throws IOException;

Review comment:
       it is better to have "close()" in BookieFileChannel 




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r693714774



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {

Review comment:
       please add a "close()" method




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-908178753


   @nicoloboschi Could you please review it again?


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r690937048



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getDeclaredConstructor().newInstance();

Review comment:
       it is better to use `getConstructor()` instead of `getDeclaredConstructor()` (includes "private" methods)

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannelProvider.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * A wrapper of FileChannel.
+ */
+public class DefaultFileChannelProvider implements FileChannelProvider{
+    @Override
+    public BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException {
+        return new DefaultFileChannel(file, configuration);
+    }
+
+    static class DefaultFileChannel implements BookieFileChannel {
+        private final File file;
+        private RandomAccessFile randomAccessFile;
+        private final ServerConfiguration configuration;
+
+        DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+            this.file = file;
+            this.configuration = serverConfiguration;
+        }
+
+        @Override
+        public FileChannel getFileChannel() throws FileNotFoundException {
+            synchronized (this) {
+                if (randomAccessFile == null) {
+                    randomAccessFile = new RandomAccessFile(file, "rw");
+                }
+                return randomAccessFile.getChannel();
+            }
+        }
+
+        @Override
+        public boolean fileExists(File file) {
+            return file.exists();
+        }
+
+        @Override
+        public FileDescriptor getFD() throws IOException {
+            synchronized (this) {
+                return randomAccessFile.getFD();

Review comment:
       NPE?
   I suggest to throw IOException if the handle is null
   Or to initialise it by calling `getFileChannel`




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] nicoloboschi commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r697395142



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
##########
@@ -868,6 +869,15 @@ public long getJournalPageCacheFlushIntervalMSec() {
         return this.getLong(JOURNAL_PAGECACHE_FLUSH_INTERVAL_MSEC, 1000);
     }
 
+    public ServerConfiguration setJournalChannelProvider(String journalChannelProvider) {

Review comment:
       javadoc for setter and getter ? 

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -180,8 +191,7 @@ private JournalChannel(File journalDirectory, long logId,
             nextPrealloc = this.preAllocSize;
             fc.write(zeros, nextPrealloc - journalAlignSize);
         } else {  // open an existing file
-            randomAccessFile = new RandomAccessFile(fn, "r");
-            fc = openFileChannel(randomAccessFile);

Review comment:
       openFileChannel method could be removed now ? 

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
##########
@@ -868,6 +869,15 @@ public long getJournalPageCacheFlushIntervalMSec() {
         return this.getLong(JOURNAL_PAGECACHE_FLUSH_INTERVAL_MSEC, 1000);
     }
 
+    public ServerConfiguration setJournalChannelProvider(String journalChannelProvider) {

Review comment:
       I feel it would be good to say that the class must implements FileChannelProvider and an no args constructor is needed

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannel.java
##########
@@ -0,0 +1,75 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+class DefaultFileChannel implements BookieFileChannel {
+    private final File file;
+    private RandomAccessFile randomAccessFile;
+    private final ServerConfiguration configuration;
+
+    DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+        this.file = file;
+        this.configuration = serverConfiguration;
+    }
+
+    @Override
+    public FileChannel getFileChannel() throws FileNotFoundException {
+        synchronized (this) {
+            if (randomAccessFile == null) {
+                randomAccessFile = new RandomAccessFile(file, "rw");
+            }
+            return randomAccessFile.getChannel();
+        }
+    }
+
+    @Override
+    public boolean fileExists(File file) {
+        return file.exists();
+    }
+
+    @Override
+    public FileDescriptor getFD() throws IOException {
+        synchronized (this) {
+            if (randomAccessFile == null) {
+                throw new IOException("randomAccessFile is null, please initialize it by calling getFileChannel");
+            }
+            return randomAccessFile.getFD();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        synchronized (this) {

Review comment:
       It looks like the close() method of RandomAccessFile wasn't called before
   and fd and channel related to the file were closed due to `BufferedChannel#close()` method
   
   However closing the RandomAccessFile twice is not a problem, so we can keep it as is




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-896068841


   rerun failure checks


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-908303322


   committed to master branch, it will be available on BK 4.15.0, hopefully very soon


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-908178196


   rerun failure checks


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-908303322


   committed to master branch, it will be available on BK 4.15.0, hopefully very soon


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r693792726



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return FileChannelProvider. A file channel provider loaded from providerClassName
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getConstructor().newInstance();
+            return (FileChannelProvider) obj;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    /**
+     * Get the BookieFileChannel with the given file and configuration.
+     *
+     * @param file
+     * @param configuration
+     * @return BookieFileChannel related to file parameter.
+     * @throws IOException
+     */
+    BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException;
+
+    void close(BookieFileChannel bookieFileChannel) throws IOException;

Review comment:
       Should I change it to "close()" and create a BookieFileChannel member opened by this provider? @eolivelli 




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r696798099



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -143,27 +149,37 @@
     private JournalChannel(File journalDirectory, long logId,
                            long preAllocSize, int writeBufferSize, int journalAlignSize,
                            long position, boolean fRemoveFromPageCache,
-                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder) throws IOException {
+                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder,
+                           ServerConfiguration configuration) throws IOException {
         this.journalAlignSize = journalAlignSize;
         this.zeros = ByteBuffer.allocate(journalAlignSize);
         this.preAllocSize = preAllocSize - preAllocSize % journalAlignSize;
         this.fRemoveFromPageCache = fRemoveFromPageCache;
+        this.configuration = configuration;
+
         File fn = new File(journalDirectory, Long.toHexString(logId) + ".txn");
+        try {
+            FileChannelProvider provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());

Review comment:
       can't we instantiate the `FileChannelProvider` only once when the `Journal` or the `Bookie` is started ?
   
   

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -143,27 +149,37 @@
     private JournalChannel(File journalDirectory, long logId,
                            long preAllocSize, int writeBufferSize, int journalAlignSize,
                            long position, boolean fRemoveFromPageCache,
-                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder) throws IOException {
+                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder,
+                           ServerConfiguration configuration) throws IOException {
         this.journalAlignSize = journalAlignSize;
         this.zeros = ByteBuffer.allocate(journalAlignSize);
         this.preAllocSize = preAllocSize - preAllocSize % journalAlignSize;
         this.fRemoveFromPageCache = fRemoveFromPageCache;
+        this.configuration = configuration;
+
         File fn = new File(journalDirectory, Long.toHexString(logId) + ".txn");
+        try {
+            FileChannelProvider provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());

Review comment:
       we should keep a reference to this  `FileChannelProvider` as a field
   and call `provider.close()` when closing the `JournalChannel`




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r698003603



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannel.java
##########
@@ -0,0 +1,75 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+class DefaultFileChannel implements BookieFileChannel {
+    private final File file;
+    private RandomAccessFile randomAccessFile;
+    private final ServerConfiguration configuration;
+
+    DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+        this.file = file;
+        this.configuration = serverConfiguration;

Review comment:
       @dlg99 Thank you for your review! Indeed some specific values should be passed to FileChannelProvider, I think it's convenient for users to add these values in server configuration. And we don't want to change the code in JournalChannel, every provider should be opened in the same way, so it is not good to pass values to the different constructors. 




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-902717856


   I would add a close() method anyway.
   You never know if some implementation needs to release additional resources.
   
   Yes I would add a test to verify that all methods are executed in the expected sequence


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: [WIP] Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r654883392



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannelProvider.java
##########
@@ -0,0 +1,64 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+
+public class DefaultFileChannelProvider implements FileChannelProvider{
+    @Override
+    public BookieFileChannel open(File file, ServerConfiguration configuration) throws IOException {
+        return new DefaultFileChannel(file, configuration);
+    }
+
+    static class DefaultFileChannel implements BookieFileChannel {
+        private final File file;
+        private static RandomAccessFile randomAccessFile;
+        private final ServerConfiguration configuration;
+
+        DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+            this.file = file;
+            this.configuration = serverConfiguration;
+        }
+
+        @Override
+        public FileChannel getFileChannel() throws FileNotFoundException {
+            if (randomAccessFile == null) {

Review comment:
       There isn't any kind of concurrency control here.
   
   Is this intended?
   IMHO it is better to add at least 'synchonized'

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -143,27 +146,39 @@
     private JournalChannel(File journalDirectory, long logId,
                            long preAllocSize, int writeBufferSize, int journalAlignSize,
                            long position, boolean fRemoveFromPageCache,
-                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder) throws IOException {
+                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder,
+                           ServerConfiguration configuration) throws IOException {
         this.journalAlignSize = journalAlignSize;
         this.zeros = ByteBuffer.allocate(journalAlignSize);
         this.preAllocSize = preAllocSize - preAllocSize % journalAlignSize;
         this.fRemoveFromPageCache = fRemoveFromPageCache;
+        this.configuration = configuration;
+
         File fn = new File(journalDirectory, Long.toHexString(logId) + ".txn");
+        FileChannelProvider provider;
+        // Create the file channel provider with given configuration. Fallback to use default FileChannel if
+        // load failed.
+        try {
+            provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());
+        } catch (IOException e) {
+            LOG.warn("Failed to load journal channel provider '{}', fallback to the default JournalChannel", configuration.getJournalChannelProvider(), e);
+            provider = new DefaultFileChannelProvider();

Review comment:
       We should fail here because in this case the Bookie won't work as expected by who configured it
   And the impact on the system will be significant 




-- 
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:
users@infra.apache.org



[GitHub] [bookkeeper] Sunny-Island commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-903418645


   @eolivelli Please review again.


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r696649280



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {

Review comment:
       I mean to also add a close() method, or better, to add "extends AutoCloseable" to the interface




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r689234563



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getDeclaredConstructor().newInstance();

Review comment:
       providerClass.newInstance() has been de




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] zymap commented on pull request #2742: [WIP] Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-882153075


   Hi @eolivelli. We have published a proposal on the mailing list, please take a look, thanks! https://lists.apache.org/thread.html/r0abbe15a9b7c501b602d44d954800217cc80268c7d67dd350187a710%40%3Cdev.bookkeeper.apache.org%3E


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] dlg99 commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
dlg99 commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r697602485



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultFileChannel.java
##########
@@ -0,0 +1,75 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+class DefaultFileChannel implements BookieFileChannel {
+    private final File file;
+    private RandomAccessFile randomAccessFile;
+    private final ServerConfiguration configuration;
+
+    DefaultFileChannel(File file, ServerConfiguration serverConfiguration) throws IOException {
+        this.file = file;
+        this.configuration = serverConfiguration;

Review comment:
       `configuration` is not used anywhere. Also it smells weird that a FileChannel (low level abstraction) depends on server (very high level abstraction) configuration.
   I suggest removing `ServerConfiguration` parameter.
   FileChannelProvider can get specific values, as needed, to pass to the constructor of specific implementation.




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r689234563



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return
+     * @throws IOException
+     */
+    static FileChannelProvider newProvider(String providerClassName) throws IOException {
+        try {
+            Class<?> providerClass = Class.forName(providerClassName);
+            Object obj = providerClass.getDeclaredConstructor().newInstance();

Review comment:
       providerClass.newInstance() has been deprecated since java 9. Bookkeeper's  CI test has Java 11 version.
   https://docs.oracle.com/javase/9/docs/api/java/lang/Class.html#newInstance--

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileChannelProvider.java
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+/**
+ * An interface of the FileChannelProvider.
+ */
+public interface FileChannelProvider {
+    /**
+     *
+     * @param providerClassName Provided class name for file channel.
+     * @return

Review comment:
       ok




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r689234993



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -85,37 +86,41 @@
     // The position of the file channel's last drop position
     private long lastDropPosition = 0L;
 
+    final ServerConfiguration configuration;
+
     // Mostly used by tests
     JournalChannel(File journalDirectory, long logId) throws IOException {
-        this(journalDirectory, logId, 4 * 1024 * 1024, 65536, START_OF_FILE);
+        this(journalDirectory, logId, 4 * 1024 * 1024, 65536, START_OF_FILE, new ServerConfiguration());

Review comment:
       ok




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r689234932



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -143,27 +148,32 @@
     private JournalChannel(File journalDirectory, long logId,
                            long preAllocSize, int writeBufferSize, int journalAlignSize,
                            long position, boolean fRemoveFromPageCache,
-                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder) throws IOException {
+                           int formatVersionToWrite, Journal.BufferedChannelBuilder bcBuilder,
+                           ServerConfiguration configuration) throws IOException {
         this.journalAlignSize = journalAlignSize;
         this.zeros = ByteBuffer.allocate(journalAlignSize);
         this.preAllocSize = preAllocSize - preAllocSize % journalAlignSize;
         this.fRemoveFromPageCache = fRemoveFromPageCache;
+        this.configuration = configuration;
+
         File fn = new File(journalDirectory, Long.toHexString(logId) + ".txn");
+        FileChannelProvider provider;
+        provider = FileChannelProvider.newProvider(configuration.getJournalChannelProvider());

Review comment:
       ok




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island edited a comment on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island edited a comment on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-902567598


   `FileChannelProvider ` do not hold any resources, maybe you mean add `close() `to BookieFileChannel? `close()` method can release `RandomAccessFile` and its `FileChannel` @eolivelli .
   Also, I guess I need to add test to ensure DefaultFileChannel methods are invoked in the correct sequence?


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] eolivelli commented on pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#issuecomment-903504157


   @zymap is there a particular reason why you are not adding a close() method to `FileChannelProvider` (you can make it extend Closable or AutoCloseable) ?
   
   For instance if the `FileChannelProvider` implements some kind of resource pooling or needs some procedure for a graceful shutdown currently there is no way to implement it.
   
   Adding a `close() ` method is very easy and it will make like easier for implementors of `FileChannelProvider`


-- 
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: issues-unsubscribe@bookkeeper.apache.org

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



[GitHub] [bookkeeper] Sunny-Island commented on a change in pull request #2742: Abstract the FileChannel in the JournalChannel

Posted by GitBox <gi...@apache.org>.
Sunny-Island commented on a change in pull request #2742:
URL: https://github.com/apache/bookkeeper/pull/2742#discussion_r697961524



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
##########
@@ -180,8 +191,7 @@ private JournalChannel(File journalDirectory, long logId,
             nextPrealloc = this.preAllocSize;
             fc.write(zeros, nextPrealloc - journalAlignSize);
         } else {  // open an existing file
-            randomAccessFile = new RandomAccessFile(fn, "r");
-            fc = openFileChannel(randomAccessFile);

Review comment:
       It is still used by some test.




-- 
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: issues-unsubscribe@bookkeeper.apache.org

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