You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2017/04/20 17:35:27 UTC

[5/7] bookkeeper git commit: Remove sync_range

Remove sync_range


Project: http://git-wip-us.apache.org/repos/asf/bookkeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/bookkeeper/commit/74113b5c
Tree: http://git-wip-us.apache.org/repos/asf/bookkeeper/tree/74113b5c
Diff: http://git-wip-us.apache.org/repos/asf/bookkeeper/diff/74113b5c

Branch: refs/heads/sijie/bookkeeper_fallocate
Commit: 74113b5cf390132d1e6ddb13ba3c207e076df758
Parents: 4292282
Author: Sijie Guo <si...@apache.org>
Authored: Tue Apr 18 11:19:24 2017 -0700
Committer: Sijie Guo <si...@apache.org>
Committed: Tue Apr 18 11:19:24 2017 -0700

----------------------------------------------------------------------
 .../org/apache/bookkeeper/bookie/Journal.java   | 11 +----
 .../bookkeeper/bookie/JournalChannel.java       | 30 --------------
 .../org/apache/bookkeeper/util/NativeIO.java    | 43 --------------------
 .../src/org/apache/bookkeeper/util/NativeIO.c   | 41 -------------------
 4 files changed, 1 insertion(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/74113b5c/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
index e3531f6..2862144 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
@@ -335,12 +335,7 @@ class Journal extends BookieCriticalThread implements CheckpointSource {
             try {
                 if (shouldForceWrite) {
                     long startTime = MathUtils.nowInNano();
-                    if (enableGroupForceWrites) {
-                        this.logFile.forceWrite(false);
-                    } else {
-                        this.logFile.syncRangeOrForceWrite(this.startFlushPosition,
-                            this.endFlushPosition - this.startFlushPosition);
-                    }
+                    this.logFile.forceWrite(false);
                     journalSyncStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS);
                 }
                 lastLogMark.setCurLogMark(this.logId, this.endFlushPosition);
@@ -932,10 +927,6 @@ class Journal extends BookieCriticalThread implements CheckpointSource {
                             bc.flush(false);
                             lastFlushPosition = bc.position();
 
-                            // start sync the range
-                            if (!enableGroupForceWrites) {
-                                logFile.startSyncRange(prevFlushPosition, lastFlushPosition);
-                            }
                             journalFlushStats.registerSuccessfulEvent(
                                     journalFlushWatcher.stop().elapsedTime(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
 

http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/74113b5c/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
index e3077e1..07e3d3d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
@@ -30,7 +30,6 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -44,7 +43,6 @@ import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Charsets.UTF_8;
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.*;
-import static org.apache.bookkeeper.util.NativeIO.*;
 
 /**
  * Simple wrapper around FileChannel to add versioning
@@ -192,9 +190,6 @@ class JournalChannel implements Closeable {
             fc.write(bb);
 
             bc = new BufferedChannel(fc, writeBufferSize);
-
-            // sync the file
-            // syncRangeOrForceWrite(0, HEADER_SIZE);
         } else {  // open an existing file
             randomAccessFile = new RandomAccessFile(fn, "r");
             fd = NativeIO.getSysFileDescriptor(randomAccessFile.getFD());
@@ -302,20 +297,6 @@ class JournalChannel implements Closeable {
         fc.close();
     }
 
-    public void startSyncRange(long offset, long bytes) throws IOException {
-        NativeIO.syncFileRangeIfPossible(fd, offset, bytes, SYNC_FILE_RANGE_WRITE);
-    }
-
-    public boolean syncRangeIfPossible(long offset, long bytes) throws IOException {
-        if (NativeIO.syncFileRangeIfPossible(fd, offset, bytes,
-                SYNC_FILE_RANGE_WAIT_BEFORE | SYNC_FILE_RANGE_WRITE | SYNC_FILE_RANGE_WAIT_AFTER)) {
-            removeFromPageCacheIfPossible(offset + bytes);
-            return false;
-        } else {
-            return false;
-        }
-    }
-
     public void forceWrite(boolean forceMetadata) throws IOException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("Journal ForceWrite");
@@ -356,15 +337,4 @@ class JournalChannel implements Closeable {
         removeFromPageCacheIfPossible(newForceWritePosition);
     }
 
-    public void syncRangeOrForceWrite(long offset, long bytes) throws IOException {
-        long startTimeNanos = MathUtils.nowInNano();
-        if (!syncRangeIfPossible(offset, bytes)) {
-            forceWriteImpl(false);
-        }
-        // collect stats
-        journalForceWriteCounter.inc();
-        journalForceWriteStats.registerSuccessfulEvent(
-                MathUtils.elapsedMicroSec(startTimeNanos),
-                TimeUnit.MICROSECONDS);
-    }
 }

http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/74113b5c/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/NativeIO.java
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/NativeIO.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/NativeIO.java
index 9eb3a68..80e0ee2 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/NativeIO.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/NativeIO.java
@@ -29,25 +29,10 @@ public final class NativeIO {
 
     private static final int POSIX_FADV_DONTNEED = 4; /* fadvise.h */
 
-    /**
-     *  Wait upon writeout of all pages in the range before performing the write.
-     */
-    public static final int SYNC_FILE_RANGE_WAIT_BEFORE = 1;
-    /**
-     * Initiate writeout of all those dirty pages in the range which are not presently
-     * under writeback.
-     */
-    public static final int SYNC_FILE_RANGE_WRITE = 2;
-    /**
-     * Wait upon writeout of all pages in the range after performing the write.
-     */
-    public static final int SYNC_FILE_RANGE_WAIT_AFTER = 4;
-
     private static final int FALLOC_FL_KEEP_SIZE = 1;
 
     private static boolean initialized = false;
     private static boolean fadvisePossible = true;
-    private static boolean syncFileRangePossible = true;
     private static boolean sysFallocatePossible = true;
     private static boolean posixFallocatePossible = true;
 
@@ -68,8 +53,6 @@ public final class NativeIO {
     public static native int posix_fallocate(int fd, long offset, long len);
     // fallocate
     public static native int fallocate(int fd, int mode, long offset, long len);
-    // sync_file_range(2)
-    public static native int sync_file_range(int fd, long offset, long len, int flags);
 
     private NativeIO() {}
 
@@ -165,32 +148,6 @@ public final class NativeIO {
         return posixFallocatePossible;
     }
 
-    public static boolean syncFileRangeIfPossible(int fd, long offset, long nbytes, int flags) {
-        if (!initialized || !syncFileRangePossible || fd < 0) {
-            return false;
-        }
-        try {
-            int rc = sync_file_range(fd, offset, nbytes, flags);
-            if (rc != 0) {
-                LOG.error("Failed on syncing file descriptor {}, offset {}, bytes {}, rc {} : {}",
-                        new Object[] { fd, offset, nbytes, rc, Errno.strerror() });
-                return false;
-            }
-        } catch (UnsupportedOperationException uoe) {
-            LOG.warn("sync_file_range isn't supported : ", uoe);
-            syncFileRangePossible = false;
-        }  catch (UnsatisfiedLinkError nle) {
-            LOG.warn("Unsatisfied Link error: sync_file_range failed on file descriptor {}, offset {}, bytes {} : ",
-                    new Object[] { fd, offset, nbytes, nle });
-            syncFileRangePossible = false;
-        } catch (Exception e) {
-            LOG.error("Unknown exception: sync_file_range failed on file descriptor {}, offset {}, bytes {} : ",
-                    new Object[] { fd, offset, nbytes, e });
-            return false;
-        }
-        return syncFileRangePossible;
-    }
-
     /**
      * Remove pages from the file system page cache when they wont
      * be accessed again

http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/74113b5c/bookkeeper-server/src/main/native/src/org/apache/bookkeeper/util/NativeIO.c
----------------------------------------------------------------------
diff --git a/bookkeeper-server/src/main/native/src/org/apache/bookkeeper/util/NativeIO.c b/bookkeeper-server/src/main/native/src/org/apache/bookkeeper/util/NativeIO.c
index b93bde4..9ba2854 100644
--- a/bookkeeper-server/src/main/native/src/org/apache/bookkeeper/util/NativeIO.c
+++ b/bookkeeper-server/src/main/native/src/org/apache/bookkeeper/util/NativeIO.c
@@ -25,47 +25,6 @@
 #include <asm-x86_64/unistd.h>
 #include "config.h"
 
-#if defined(HAVE_SYNC_FILE_RANGE)
-#  define my_sync_file_range sync_file_range
-#elif defined(__NR_sync_file_range)
-// RHEL 5 kernels have sync_file_range support, but the glibc
-// included does not have the library function. We can
-// still call it directly, and if it's not supported by the
-// kernel, we'd get ENOSYS. See RedHat Bugzilla #518581
-static int manual_sync_file_range (int fd, __off64_t from, __off64_t to, unsigned int flags)
-{
-#ifdef __x86_64__
-  return syscall( __NR_sync_file_range, fd, from, to, flags);
-#else
-  return syscall (__NR_sync_file_range, fd,
-    __LONG_LONG_PAIR ((long) (from >> 32), (long) from),
-    __LONG_LONG_PAIR ((long) (to >> 32), (long) to),
-    flags);
-#endif
-}
-#define my_sync_file_range manual_sync_file_range
-#endif
-
-/**
- * public static native void sync_file_range(
- *   int fd, long offset, long len, int flags);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jint JNICALL
-Java_org_apache_bookkeeper_util_NativeIO_sync_1file_1range(
-  JNIEnv *env, jclass clazz,
-  jint fd, jlong offset, jlong len, jint flags)
-{
-#ifndef my_sync_file_range
-  errno = ENOSYS;
-  return -1;
-#else
-  return my_sync_file_range(fd, (off_t)offset, (off_t)len, flags);
-#endif
-}
-
 #if defined(HAVE_FALLOCATE)
 #  define my_fallocate fallocate
 #elif defined(__NR_fallocate)