You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2017/03/28 14:42:57 UTC

[3/6] cassandra git commit: Use the Kernel32 library to retrieve the PID on Windows and fix startup checks

Use the Kernel32 library to retrieve the PID on Windows and fix startup checks

patch by Benjamin Lerer; reviewed by Michael Kjellman for CASSANDRA-13333


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9b8692c6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9b8692c6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9b8692c6

Branch: refs/heads/trunk
Commit: 9b8692c6a4c75b7df29a58b5d3d1d1ee5cb0c3a4
Parents: 6da41ed
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Mar 28 16:09:59 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Mar 28 16:17:10 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +
 .../db/commitlog/CommitLogSegment.java          |   4 +-
 .../db/commitlog/MemoryMappedSegment.java       |   4 +-
 .../cassandra/db/lifecycle/LogReplica.java      |  10 +-
 .../apache/cassandra/hints/HintsCatalog.java    |   6 +-
 .../org/apache/cassandra/hints/HintsReader.java |   4 +-
 .../org/apache/cassandra/hints/HintsWriter.java |   6 +-
 .../cassandra/io/sstable/SSTableRewriter.java   |   4 +-
 .../io/sstable/format/SSTableReader.java        |   4 +-
 .../apache/cassandra/io/util/ChannelProxy.java  |   4 +-
 .../apache/cassandra/io/util/SegmentedFile.java |   4 +-
 .../cassandra/service/CassandraDaemon.java      |   6 +-
 .../cassandra/service/NativeAccessMBean.java    |   8 +
 .../apache/cassandra/service/StartupChecks.java |  10 +-
 .../org/apache/cassandra/utils/CLibrary.java    | 396 ------------------
 .../apache/cassandra/utils/CLibraryDarwin.java  | 131 ------
 .../apache/cassandra/utils/CLibraryLinux.java   | 131 ------
 .../apache/cassandra/utils/CLibraryWindows.java | 103 -----
 .../apache/cassandra/utils/CLibraryWrapper.java |  40 --
 .../org/apache/cassandra/utils/HeapUtils.java   |   2 +-
 .../apache/cassandra/utils/NativeLibrary.java   | 400 +++++++++++++++++++
 .../cassandra/utils/NativeLibraryDarwin.java    | 129 ++++++
 .../cassandra/utils/NativeLibraryLinux.java     | 129 ++++++
 .../cassandra/utils/NativeLibraryWindows.java   | 124 ++++++
 .../cassandra/utils/NativeLibraryWrapper.java   |  44 ++
 .../org/apache/cassandra/utils/SyncUtil.java    |   6 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   2 +-
 .../apache/cassandra/utils/WindowsTimer.java    |   4 +
 .../apache/cassandra/utils/CLibraryTest.java    |  44 --
 .../cassandra/utils/NativeLibraryTest.java      |  44 ++
 31 files changed, 930 insertions(+), 883 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0af7a7d..ca79a01 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.13
+ * Use the Kernel32 library to retrieve the PID on Windows and fix startup checks (CASSANDRA-13333)
  * Fix code to not exchange schema across major versions (CASSANDRA-13274)
  * Dropping column results in "corrupt" SSTable (CASSANDRA-13337)
  * Bugs handling range tombstones in the sstable iterators (CASSANDRA-13340)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index faba342..b6faef4 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,6 +13,15 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
+3.0.13
+======
+
+Upgrading
+---------
+   - The NativeAccessMBean isAvailable method will only return true if the
+     native library has been successfully linked. Previously it was returning
+     true if JNA could be found but was not taking into account link failures.
+
 3.0.12
 ======
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 0a03c3c..f26f0dc 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -48,7 +48,7 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
@@ -156,7 +156,7 @@ public abstract class CommitLogSegment
         try
         {
             channel = FileChannel.open(logFile.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE);
-            fd = CLibrary.getfd(channel);
+            fd = NativeLibrary.getfd(channel);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
index afc8dcc..326469f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
@@ -25,7 +25,7 @@ import java.nio.channels.FileChannel;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.SyncUtil;
 
 /*
@@ -86,7 +86,7 @@ public class MemoryMappedSegment extends CommitLogSegment
         {
             throw new FSWriteError(e, getPath());
         }
-        CLibrary.trySkipCache(fd, startMarker, nextMarker, logFile.getAbsolutePath());
+        NativeLibrary.trySkipCache(fd, startMarker, nextMarker, logFile.getAbsolutePath());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
index 712a22d..44400d1 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplica.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db.lifecycle;
 import java.io.File;
 
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 
 /**
  * Because a column family may have sstables on different disks and disks can
@@ -41,12 +41,12 @@ final class LogReplica implements AutoCloseable
 
     static LogReplica create(File folder, String fileName)
     {
-        return new LogReplica(new File(fileName), CLibrary.tryOpenDirectory(folder.getPath()));
+        return new LogReplica(new File(fileName), NativeLibrary.tryOpenDirectory(folder.getPath()));
     }
 
     static LogReplica open(File file)
     {
-        return new LogReplica(file, CLibrary.tryOpenDirectory(file.getParentFile().getPath()));
+        return new LogReplica(file, NativeLibrary.tryOpenDirectory(file.getParentFile().getPath()));
     }
 
     LogReplica(File file, int folderDescriptor)
@@ -74,7 +74,7 @@ final class LogReplica implements AutoCloseable
     void syncFolder()
     {
         if (folderDescriptor >= 0)
-            CLibrary.trySync(folderDescriptor);
+            NativeLibrary.trySync(folderDescriptor);
     }
 
     void delete()
@@ -92,7 +92,7 @@ final class LogReplica implements AutoCloseable
     {
         if (folderDescriptor >= 0)
         {
-            CLibrary.tryCloseFD(folderDescriptor);
+            NativeLibrary.tryCloseFD(folderDescriptor);
             folderDescriptor = -1;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/hints/HintsCatalog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsCatalog.java b/src/java/org/apache/cassandra/hints/HintsCatalog.java
index c2f0972..5ebe65b 100644
--- a/src/java/org/apache/cassandra/hints/HintsCatalog.java
+++ b/src/java/org/apache/cassandra/hints/HintsCatalog.java
@@ -27,7 +27,7 @@ import java.util.stream.Stream;
 import com.google.common.collect.ImmutableMap;
 
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.SyncUtil;
 
 import static java.util.stream.Collectors.groupingBy;
@@ -130,11 +130,11 @@ final class HintsCatalog
 
     void fsyncDirectory()
     {
-        int fd = CLibrary.tryOpenDirectory(hintsDirectory.getAbsolutePath());
+        int fd = NativeLibrary.tryOpenDirectory(hintsDirectory.getAbsolutePath());
         if (fd != -1)
         {
             SyncUtil.trySync(fd);
-            CLibrary.tryCloseFD(fd);
+            NativeLibrary.tryCloseFD(fd);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/hints/HintsReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsReader.java b/src/java/org/apache/cassandra/hints/HintsReader.java
index ef24aca..d88c4f5 100644
--- a/src/java/org/apache/cassandra/hints/HintsReader.java
+++ b/src/java/org/apache/cassandra/hints/HintsReader.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 
 /**
  * A paged non-compressed hints reader that provides two iterators:
@@ -148,7 +148,7 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
         @SuppressWarnings("resource")
         protected Page computeNext()
         {
-            CLibrary.trySkipCache(input.getChannel().getFileDescriptor(), 0, input.getFilePointer(), input.getPath());
+            NativeLibrary.trySkipCache(input.getChannel().getFileDescriptor(), 0, input.getFilePointer(), input.getPath());
 
             if (input.isEOF())
                 return endOfData();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/hints/HintsWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsWriter.java b/src/java/org/apache/cassandra/hints/HintsWriter.java
index 8836258..31a440d 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriter.java
@@ -34,7 +34,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.SyncUtil;
 import org.apache.cassandra.utils.Throwables;
 
@@ -71,7 +71,7 @@ class HintsWriter implements AutoCloseable
         File file = new File(directory, descriptor.fileName());
 
         FileChannel channel = FileChannel.open(file.toPath(), StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
-        int fd = CLibrary.getfd(channel);
+        int fd = NativeLibrary.getfd(channel);
 
         CRC32 crc = new CRC32();
 
@@ -293,7 +293,7 @@ class HintsWriter implements AutoCloseable
             // don't skip page cache for tiny files, on the assumption that if they are tiny, the target node is probably
             // alive, and if so, the file will be closed and dispatched shortly (within a minute), and the file will be dropped.
             if (position >= DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024L)
-                CLibrary.trySkipCache(fd, 0, position - (position % PAGE_SIZE), file.getPath());
+                NativeLibrary.trySkipCache(fd, 0, position - (position % PAGE_SIZE), file.getPath());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index f4a2e1b..0ea28d7 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
 /**
@@ -164,7 +164,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
                 for (SSTableReader reader : transaction.originals())
                 {
                     RowIndexEntry index = reader.getPosition(key, SSTableReader.Operator.GE);
-                    CLibrary.trySkipCache(reader.getFilename(), 0, index == null ? 0 : index.position);
+                    NativeLibrary.trySkipCache(reader.getFilename(), 0, index == null ? 0 : index.position);
                 }
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 3283723..8be1fe2 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2223,8 +2223,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                 obsoletion.run();
 
             // don't ideally want to dropPageCache for the file until all instances have been released
-            CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
-            CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);
+            NativeLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
+            NativeLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);
         }
 
         public String name()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/io/util/ChannelProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/ChannelProxy.java b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
index f866160..1463fdd 100644
--- a/src/java/org/apache/cassandra/io/util/ChannelProxy.java
+++ b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
@@ -26,7 +26,7 @@ import java.nio.channels.WritableByteChannel;
 import java.nio.file.StandardOpenOption;
 
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.concurrent.RefCounted;
 import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
 
@@ -171,7 +171,7 @@ public final class ChannelProxy extends SharedCloseableImpl
 
     public int getFileDescriptor()
     {
-        return CLibrary.getfd(channel);
+        return NativeLibrary.getfd(channel);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index ab2d291..9df4c81 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -34,7 +34,7 @@ import org.apache.cassandra.io.sstable.IndexSummary;
 import org.apache.cassandra.io.sstable.IndexSummaryBuilder;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.NativeLibrary;
 import org.apache.cassandra.utils.concurrent.RefCounted;
 import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
 
@@ -137,7 +137,7 @@ public abstract class SegmentedFile extends SharedCloseableImpl
 
     public void dropPageCache(long before)
     {
-        CLibrary.trySkipCache(channel.getFileDescriptor(), 0, before, path());
+        NativeLibrary.trySkipCache(channel.getFileDescriptor(), 0, before, path());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index b4fdd19..d267e21 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -172,7 +172,7 @@ public class CassandraDaemon
 
         logSystemInfo();
 
-        CLibrary.tryMlockall();
+        NativeLibrary.tryMlockall();
 
         try
         {
@@ -717,12 +717,12 @@ public class CassandraDaemon
     {
         public boolean isAvailable()
         {
-            return CLibrary.jnaAvailable();
+            return NativeLibrary.isAvailable();
         }
 
         public boolean isMemoryLockable()
         {
-            return CLibrary.jnaMemoryLockable();
+            return NativeLibrary.jnaMemoryLockable();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/service/NativeAccessMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/NativeAccessMBean.java b/src/java/org/apache/cassandra/service/NativeAccessMBean.java
index b0c408c..0128369 100644
--- a/src/java/org/apache/cassandra/service/NativeAccessMBean.java
+++ b/src/java/org/apache/cassandra/service/NativeAccessMBean.java
@@ -19,7 +19,15 @@ package org.apache.cassandra.service;
 
 public interface NativeAccessMBean 
 {
+    /**
+     * Checks if the native library has been successfully linked.
+     * @return {@code true} if the library has been successfully linked, {@code false} otherwise.
+     */
     boolean isAvailable();
 
+    /**
+     * Checks if the native library is able to lock memory.
+     * @return {@code true} if the native library is able to lock memory, {@code false} otherwise.
+     */
     boolean isMemoryLockable();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 756799e..19b6620 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -74,7 +74,7 @@ public class StartupChecks
                                                                       checkValidLaunchDate,
                                                                       checkJMXPorts,
                                                                       inspectJvmOptions,
-                                                                      checkJnaInitialization,
+                                                                      checkNativeLibraryInitialization,
                                                                       initSigarLibrary,
                                                                       checkMaxMapCount,
                                                                       checkDataDirs,
@@ -184,13 +184,13 @@ public class StartupChecks
         }
     };
 
-    public static final StartupCheck checkJnaInitialization = new StartupCheck()
+    public static final StartupCheck checkNativeLibraryInitialization = new StartupCheck()
     {
         public void execute() throws StartupException
         {
-            // Fail-fast if JNA is not available or failing to initialize properly
-            if (!CLibrary.jnaAvailable())
-                throw new StartupException(3, "JNA failing to initialize properly. ");
+            // Fail-fast if the native library could not be linked.
+            if (!NativeLibrary.isAvailable())
+                throw new StartupException(3, "The native library could not be initialized properly. ");
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/CLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
deleted file mode 100644
index 71f4148..0000000
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ /dev/null
@@ -1,396 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import java.io.File;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jna.LastErrorException;
-
-import static org.apache.cassandra.utils.CLibrary.OSType.LINUX;
-import static org.apache.cassandra.utils.CLibrary.OSType.MAC;
-import static org.apache.cassandra.utils.CLibrary.OSType.WINDOWS;
-import static org.apache.cassandra.utils.CLibrary.OSType.AIX;
-
-public final class CLibrary
-{
-    private static final Logger logger = LoggerFactory.getLogger(CLibrary.class);
-
-    public enum OSType
-    {
-        LINUX,
-        MAC,
-        WINDOWS,
-        AIX,
-        OTHER;
-    }
-
-    private static final OSType osType;
-
-    private static final int MCL_CURRENT;
-    private static final int MCL_FUTURE;
-
-    private static final int ENOMEM = 12;
-
-    private static final int F_GETFL   = 3;  /* get file status flags */
-    private static final int F_SETFL   = 4;  /* set file status flags */
-    private static final int F_NOCACHE = 48; /* Mac OS X specific flag, turns cache on/off */
-    private static final int O_DIRECT  = 040000; /* fcntl.h */
-    private static final int O_RDONLY  = 00000000; /* fcntl.h */
-
-    private static final int POSIX_FADV_NORMAL     = 0; /* fadvise.h */
-    private static final int POSIX_FADV_RANDOM     = 1; /* fadvise.h */
-    private static final int POSIX_FADV_SEQUENTIAL = 2; /* fadvise.h */
-    private static final int POSIX_FADV_WILLNEED   = 3; /* fadvise.h */
-    private static final int POSIX_FADV_DONTNEED   = 4; /* fadvise.h */
-    private static final int POSIX_FADV_NOREUSE    = 5; /* fadvise.h */
-
-    private static final CLibraryWrapper wrappedCLibrary;
-    private static boolean jnaLockable = false;
-
-    static
-    {
-        // detect the OS type the JVM is running on and then set the CLibraryWrapper
-        // instance to a compatable implementation of CLibraryWrapper for that OS type
-        osType = getOsType();
-        switch (osType)
-        {
-            case MAC: wrappedCLibrary = new CLibraryDarwin(); break;
-            case WINDOWS: wrappedCLibrary = new CLibraryWindows(); break;
-            case LINUX:
-            case AIX:
-            case OTHER:
-            default: wrappedCLibrary = new CLibraryLinux();
-        }
-
-        if (System.getProperty("os.arch").toLowerCase().contains("ppc"))
-        {
-            if (osType == LINUX)
-            {
-               MCL_CURRENT = 0x2000;
-               MCL_FUTURE = 0x4000;
-            }
-            else if (osType == AIX)
-            {
-                MCL_CURRENT = 0x100;
-                MCL_FUTURE = 0x200;
-            }
-            else
-            {
-                MCL_CURRENT = 1;
-                MCL_FUTURE = 2;
-            }
-        }
-        else
-        {
-            MCL_CURRENT = 1;
-            MCL_FUTURE = 2;
-        }
-    }
-
-    private CLibrary() {}
-
-    /**
-     * @return the detected OSType of the Operating System running the JVM using crude string matching
-     */
-    private static OSType getOsType()
-    {
-        String osName = System.getProperty("os.name").toLowerCase();
-        if (osName.contains("mac"))
-            return MAC;
-        else if (osName.contains("windows"))
-            return WINDOWS;
-        else if (osName.contains("aix"))
-            return AIX;
-        else
-            // fall back to the Linux impl for all unknown OS types until otherwise implicitly supported as needed
-            return LINUX;
-    }
-
-    private static int errno(RuntimeException e)
-    {
-        assert e instanceof LastErrorException;
-        try
-        {
-            return ((LastErrorException) e).getErrorCode();
-        }
-        catch (NoSuchMethodError x)
-        {
-            logger.warn("Obsolete version of JNA present; unable to read errno. Upgrade to JNA 3.2.7 or later");
-            return 0;
-        }
-    }
-
-    public static boolean jnaAvailable()
-    {
-        return wrappedCLibrary.jnaAvailable();
-    }
-
-    public static boolean jnaMemoryLockable()
-    {
-        return jnaLockable;
-    }
-
-    public static void tryMlockall()
-    {
-        try
-        {
-            wrappedCLibrary.callMlockall(MCL_CURRENT);
-            jnaLockable = true;
-            logger.info("JNA mlockall successful");
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // this will have already been logged by CLibrary, no need to repeat it
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            if (errno(e) == ENOMEM && osType == LINUX)
-            {
-                logger.warn("Unable to lock JVM memory (ENOMEM)."
-                        + " This can result in part of the JVM being swapped out, especially with mmapped I/O enabled."
-                        + " Increase RLIMIT_MEMLOCK or run Cassandra as root.");
-            }
-            else if (osType != MAC)
-            {
-                // OS X allows mlockall to be called, but always returns an error
-                logger.warn("Unknown mlockall error {}", errno(e));
-            }
-        }
-    }
-
-    public static void trySkipCache(String path, long offset, long len)
-    {
-        File f = new File(path);
-        if (!f.exists())
-            return;
-
-        try (FileInputStream fis = new FileInputStream(f))
-        {
-            trySkipCache(getfd(fis.getChannel()), offset, len, path);
-        }
-        catch (IOException e)
-        {
-            logger.warn("Could not skip cache", e);
-        }
-    }
-
-    public static void trySkipCache(int fd, long offset, long len, String path)
-    {
-        if (len == 0)
-            trySkipCache(fd, 0, 0, path);
-
-        while (len > 0)
-        {
-            int sublen = (int) Math.min(Integer.MAX_VALUE, len);
-            trySkipCache(fd, offset, sublen, path);
-            len -= sublen;
-            offset -= sublen;
-        }
-    }
-
-    public static void trySkipCache(int fd, long offset, int len, String path)
-    {
-        if (fd < 0)
-            return;
-
-        try
-        {
-            if (osType == LINUX)
-            {
-                int result = wrappedCLibrary.callPosixFadvise(fd, offset, len, POSIX_FADV_DONTNEED);
-                if (result != 0)
-                    NoSpamLogger.log(
-                            logger,
-                            NoSpamLogger.Level.WARN,
-                            10,
-                            TimeUnit.MINUTES,
-                            "Failed trySkipCache on file: {} Error: " + wrappedCLibrary.callStrerror(result).getString(0),
-                            path);
-            }
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // if JNA is unavailable just skipping Direct I/O
-            // instance of this class will act like normal RandomAccessFile
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            logger.warn(String.format("posix_fadvise(%d, %d) failed, errno (%d).", fd, offset, errno(e)));
-        }
-    }
-
-    public static int tryFcntl(int fd, int command, int flags)
-    {
-        // fcntl return value may or may not be useful, depending on the command
-        int result = -1;
-
-        try
-        {
-            result = wrappedCLibrary.callFcntl(fd, command, flags);
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // if JNA is unavailable just skipping
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            logger.warn(String.format("fcntl(%d, %d, %d) failed, errno (%d).", fd, command, flags, errno(e)));
-        }
-
-        return result;
-    }
-
-    public static int tryOpenDirectory(String path)
-    {
-        int fd = -1;
-
-        try
-        {
-            return wrappedCLibrary.callOpen(path, O_RDONLY);
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // JNA is unavailable just skipping Direct I/O
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            logger.warn(String.format("open(%s, O_RDONLY) failed, errno (%d).", path, errno(e)));
-        }
-
-        return fd;
-    }
-
-    public static void trySync(int fd)
-    {
-        if (fd == -1)
-            return;
-
-        try
-        {
-            wrappedCLibrary.callFsync(fd);
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // JNA is unavailable just skipping Direct I/O
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            logger.warn("fsync({}) failed, errorno ({}) {}", fd, errno(e), e);
-        }
-    }
-
-    public static void tryCloseFD(int fd)
-    {
-        if (fd == -1)
-            return;
-
-        try
-        {
-            wrappedCLibrary.callClose(fd);
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            // JNA is unavailable just skipping Direct I/O
-        }
-        catch (RuntimeException e)
-        {
-            if (!(e instanceof LastErrorException))
-                throw e;
-
-            logger.warn(String.format("close(%d) failed, errno (%d).", fd, errno(e)));
-        }
-    }
-
-    public static int getfd(FileChannel channel)
-    {
-        Field field = FBUtilities.getProtectedField(channel.getClass(), "fd");
-
-        try
-        {
-            return getfd((FileDescriptor)field.get(channel));
-        }
-        catch (IllegalArgumentException|IllegalAccessException e)
-        {
-            logger.warn("Unable to read fd field from FileChannel");
-        }
-        return -1;
-    }
-
-    /**
-     * Get system file descriptor from FileDescriptor object.
-     * @param descriptor - FileDescriptor objec to get fd from
-     * @return file descriptor, -1 or error
-     */
-    public static int getfd(FileDescriptor descriptor)
-    {
-        Field field = FBUtilities.getProtectedField(descriptor.getClass(), "fd");
-
-        try
-        {
-            return field.getInt(descriptor);
-        }
-        catch (Exception e)
-        {
-            JVMStabilityInspector.inspectThrowable(e);
-            logger.warn("Unable to read fd field from FileDescriptor");
-        }
-
-        return -1;
-    }
-
-    /**
-     * @return the PID of the JVM or -1 if we failed to get the PID
-     */
-    public static long getProcessID()
-    {
-        try
-        {
-            return wrappedCLibrary.callGetpid();
-        }
-        catch (Exception e)
-        {
-            logger.info("Failed to get PID from JNA", e);
-        }
-
-        return -1;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/CLibraryDarwin.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibraryDarwin.java b/src/java/org/apache/cassandra/utils/CLibraryDarwin.java
deleted file mode 100644
index e0a43ec..0000000
--- a/src/java/org/apache/cassandra/utils/CLibraryDarwin.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jna.LastErrorException;
-import com.sun.jna.Native;
-import com.sun.jna.Pointer;
-
-/**
- * A CLibraryWrapper implementation for Darwin/Mac.
- * <p>
- * When JNA is initialized, all methods that have the 'native' keyword
- * will be attmpted to be linked against. As Java doesn't have the equivalent
- * of a #ifdef, this means if a native method like posix_fadvise is defined in the
- * class but not available on the target operating system (e.g.
- * posix_fadvise is not availble on Darwin/Mac) this will cause the entire
- * initial linking and initialization of JNA to fail. This means other
- * native calls that are supported on that target operating system will be
- * unavailable simply because of one native defined method not supported
- * on the runtime operating system.
- * @see org.apache.cassandra.utils.CLibraryWrapper
- * @see CLibrary
- */
-public class CLibraryDarwin implements CLibraryWrapper
-{
-    private static final Logger logger = LoggerFactory.getLogger(CLibraryDarwin.class);
-
-    private static boolean jnaAvailable = true;
-
-    static
-    {
-        try
-        {
-            Native.register("c");
-        }
-        catch (NoClassDefFoundError e)
-        {
-            logger.warn("JNA not found. Native methods will be disabled.");
-            jnaAvailable = false;
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            logger.warn("JNA link failure, one or more native method will be unavailable.");
-            logger.error("JNA link failure details: {}", e.getMessage());
-        }
-        catch (NoSuchMethodError e)
-        {
-            logger.warn("Obsolete version of JNA present; unable to register C library. Upgrade to JNA 3.2.7 or later");
-            jnaAvailable = false;
-        }
-    }
-
-    private static native int mlockall(int flags) throws LastErrorException;
-    private static native int munlockall() throws LastErrorException;
-    private static native int fcntl(int fd, int command, long flags) throws LastErrorException;
-    private static native int open(String path, int flags) throws LastErrorException;
-    private static native int fsync(int fd) throws LastErrorException;
-    private static native int close(int fd) throws LastErrorException;
-    private static native Pointer strerror(int errnum) throws LastErrorException;
-    private static native long getpid() throws LastErrorException;
-
-    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return mlockall(flags);
-    }
-
-    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
-    {
-        return munlockall();
-    }
-
-    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return fcntl(fd, command, flags);
-    }
-
-    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
-    {
-        // posix_fadvise is not available on Darwin/Mac
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return open(path, flags);
-    }
-
-    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return fsync(fd);
-    }
-
-    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return close(fd);
-    }
-
-    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return strerror(errnum);
-    }
-
-    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
-    {
-        return getpid();
-    }
-
-    public boolean jnaAvailable()
-    {
-        return jnaAvailable;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/CLibraryLinux.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibraryLinux.java b/src/java/org/apache/cassandra/utils/CLibraryLinux.java
deleted file mode 100644
index 1822bdf..0000000
--- a/src/java/org/apache/cassandra/utils/CLibraryLinux.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jna.LastErrorException;
-import com.sun.jna.Native;
-import com.sun.jna.Pointer;
-
-/**
- * A CLibraryWrapper implementation for Linux.
- * <p>
- * When JNA is initialized, all methods that have the 'native' keyword
- * will be attmpted to be linked against. As Java doesn't have the equivalent
- * of a #ifdef, this means if a native method like posix_fadvise is defined in the
- * class but not available on the target operating system (e.g.
- * posix_fadvise is not availble on Darwin/Mac) this will cause the entire
- * initial linking and initialization of JNA to fail. This means other
- * native calls that are supported on that target operating system will be
- * unavailable simply because of one native defined method not supported
- * on the runtime operating system.
- * @see org.apache.cassandra.utils.CLibraryWrapper
- * @see CLibrary
- */
-public class CLibraryLinux implements CLibraryWrapper
-{
-    private static boolean jnaAvailable = true;
-
-    private static final Logger logger = LoggerFactory.getLogger(CLibraryLinux.class);
-
-    static
-    {
-        try
-        {
-            Native.register("c");
-        }
-        catch (NoClassDefFoundError e)
-        {
-            logger.warn("JNA not found. Native methods will be disabled.");
-            jnaAvailable = false;
-        }
-        catch (UnsatisfiedLinkError e)
-        {
-            logger.warn("JNA link failure, one or more native method will be unavailable.");
-            logger.error("JNA link failure details: {}", e.getMessage());
-        }
-        catch (NoSuchMethodError e)
-        {
-            logger.warn("Obsolete version of JNA present; unable to register C library. Upgrade to JNA 3.2.7 or later");
-            jnaAvailable = false;
-        }
-    }
-
-    private static native int mlockall(int flags) throws LastErrorException;
-    private static native int munlockall() throws LastErrorException;
-    private static native int fcntl(int fd, int command, long flags) throws LastErrorException;
-    private static native int posix_fadvise(int fd, long offset, int len, int flag) throws LastErrorException;
-    private static native int open(String path, int flags) throws LastErrorException;
-    private static native int fsync(int fd) throws LastErrorException;
-    private static native int close(int fd) throws LastErrorException;
-    private static native Pointer strerror(int errnum) throws LastErrorException;
-    private static native long getpid() throws LastErrorException;
-
-    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return mlockall(flags);
-    }
-
-    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
-    {
-        return munlockall();
-    }
-
-    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return fcntl(fd, command, flags);
-    }
-
-    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return posix_fadvise(fd, offset, len, flag);
-    }
-
-    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return open(path, flags);
-    }
-
-    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return fsync(fd);
-    }
-
-    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return close(fd);
-    }
-
-    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
-    {
-        return strerror(errnum);
-    }
-
-    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
-    {
-        return getpid();
-    }
-
-    public boolean jnaAvailable()
-    {
-        return jnaAvailable;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/CLibraryWindows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibraryWindows.java b/src/java/org/apache/cassandra/utils/CLibraryWindows.java
deleted file mode 100644
index c8b5bbb..0000000
--- a/src/java/org/apache/cassandra/utils/CLibraryWindows.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jna.Pointer;
-
-/**
- * A CLibraryWrapper implementation for Windows.
- * <p>
- * As libc isn't available on Windows these implementations
- * will obviously be a no-op, however when possible implementations
- * are used that are Windows friendly that will return the same
- * return value.
- * @see org.apache.cassandra.utils.CLibraryWrapper
- * @see CLibrary
- */
-public class CLibraryWindows implements CLibraryWrapper
-{
-    private static final Logger logger = LoggerFactory.getLogger(CLibraryWindows.class);
-
-    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
-    {
-        throw new UnsatisfiedLinkError();
-    }
-
-    /**
-     * @return the PID of the JVM running
-     * @throws UnsatisfiedLinkError if we fail to link against Sigar
-     * @throws RuntimeException if another unexpected error is thrown by Sigar
-     */
-    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
-    {
-        try
-        {
-            return SigarLibrary.instance.getPid();
-        }
-        catch (Exception e)
-        {
-            logger.error("Failed to initialize or use Sigar Library", e);
-        }
-
-        return -1;
-    }
-
-    public boolean jnaAvailable()
-    {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/CLibraryWrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CLibraryWrapper.java b/src/java/org/apache/cassandra/utils/CLibraryWrapper.java
deleted file mode 100644
index b97fa64..0000000
--- a/src/java/org/apache/cassandra/utils/CLibraryWrapper.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import com.sun.jna.Pointer;
-
-/**
- * An interface to implement for using OS specific native methods.
- * @see CLibrary
- */
-interface CLibraryWrapper
-{
-    boolean jnaAvailable();
-
-    int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException;
-    int callMunlockall() throws UnsatisfiedLinkError, RuntimeException;
-    int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException;
-    int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException;
-    int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException;
-    int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException;
-    int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException;
-    Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException;
-    long callGetpid() throws UnsatisfiedLinkError, RuntimeException;
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java
index 34b1b58..67d7d79 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -168,7 +168,7 @@ public final class HeapUtils
      */
     private static Long getProcessId()
     {
-        long pid = CLibrary.getProcessID();
+        long pid = NativeLibrary.getProcessID();
         if (pid >= 0)
             return pid;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/NativeLibrary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibrary.java b/src/java/org/apache/cassandra/utils/NativeLibrary.java
new file mode 100644
index 0000000..f96859e
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeLibrary.java
@@ -0,0 +1,400 @@
+/*
+ * 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.cassandra.utils;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jna.LastErrorException;
+
+import static org.apache.cassandra.utils.NativeLibrary.OSType.LINUX;
+import static org.apache.cassandra.utils.NativeLibrary.OSType.MAC;
+import static org.apache.cassandra.utils.NativeLibrary.OSType.WINDOWS;
+import static org.apache.cassandra.utils.NativeLibrary.OSType.AIX;
+
+public final class NativeLibrary
+{
+    private static final Logger logger = LoggerFactory.getLogger(NativeLibrary.class);
+
+    public enum OSType
+    {
+        LINUX,
+        MAC,
+        WINDOWS,
+        AIX,
+        OTHER;
+    }
+
+    private static final OSType osType;
+
+    private static final int MCL_CURRENT;
+    private static final int MCL_FUTURE;
+
+    private static final int ENOMEM = 12;
+
+    private static final int F_GETFL   = 3;  /* get file status flags */
+    private static final int F_SETFL   = 4;  /* set file status flags */
+    private static final int F_NOCACHE = 48; /* Mac OS X specific flag, turns cache on/off */
+    private static final int O_DIRECT  = 040000; /* fcntl.h */
+    private static final int O_RDONLY  = 00000000; /* fcntl.h */
+
+    private static final int POSIX_FADV_NORMAL     = 0; /* fadvise.h */
+    private static final int POSIX_FADV_RANDOM     = 1; /* fadvise.h */
+    private static final int POSIX_FADV_SEQUENTIAL = 2; /* fadvise.h */
+    private static final int POSIX_FADV_WILLNEED   = 3; /* fadvise.h */
+    private static final int POSIX_FADV_DONTNEED   = 4; /* fadvise.h */
+    private static final int POSIX_FADV_NOREUSE    = 5; /* fadvise.h */
+
+    private static final NativeLibraryWrapper wrappedLibrary;
+    private static boolean jnaLockable = false;
+
+    static
+    {
+        // detect the OS type the JVM is running on and then set the CLibraryWrapper
+        // instance to a compatable implementation of CLibraryWrapper for that OS type
+        osType = getOsType();
+        switch (osType)
+        {
+            case MAC: wrappedLibrary = new NativeLibraryDarwin(); break;
+            case WINDOWS: wrappedLibrary = new NativeLibraryWindows(); break;
+            case LINUX:
+            case AIX:
+            case OTHER:
+            default: wrappedLibrary = new NativeLibraryLinux();
+        }
+
+        if (System.getProperty("os.arch").toLowerCase().contains("ppc"))
+        {
+            if (osType == LINUX)
+            {
+               MCL_CURRENT = 0x2000;
+               MCL_FUTURE = 0x4000;
+            }
+            else if (osType == AIX)
+            {
+                MCL_CURRENT = 0x100;
+                MCL_FUTURE = 0x200;
+            }
+            else
+            {
+                MCL_CURRENT = 1;
+                MCL_FUTURE = 2;
+            }
+        }
+        else
+        {
+            MCL_CURRENT = 1;
+            MCL_FUTURE = 2;
+        }
+    }
+
+    private NativeLibrary() {}
+
+    /**
+     * @return the detected OSType of the Operating System running the JVM using crude string matching
+     */
+    private static OSType getOsType()
+    {
+        String osName = System.getProperty("os.name").toLowerCase();
+        if (osName.contains("mac"))
+            return MAC;
+        else if (osName.contains("windows"))
+            return WINDOWS;
+        else if (osName.contains("aix"))
+            return AIX;
+        else
+            // fall back to the Linux impl for all unknown OS types until otherwise implicitly supported as needed
+            return LINUX;
+    }
+
+    private static int errno(RuntimeException e)
+    {
+        assert e instanceof LastErrorException;
+        try
+        {
+            return ((LastErrorException) e).getErrorCode();
+        }
+        catch (NoSuchMethodError x)
+        {
+            logger.warn("Obsolete version of JNA present; unable to read errno. Upgrade to JNA 3.2.7 or later");
+            return 0;
+        }
+    }
+
+    /**
+     * Checks if the library has been successfully linked.
+     * @return {@code true} if the library has been successfully linked, {@code false} otherwise.
+     */
+    public static boolean isAvailable()
+    {
+        return wrappedLibrary.isAvailable();
+    }
+
+    public static boolean jnaMemoryLockable()
+    {
+        return jnaLockable;
+    }
+
+    public static void tryMlockall()
+    {
+        try
+        {
+            wrappedLibrary.callMlockall(MCL_CURRENT);
+            jnaLockable = true;
+            logger.info("JNA mlockall successful");
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // this will have already been logged by CLibrary, no need to repeat it
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            if (errno(e) == ENOMEM && osType == LINUX)
+            {
+                logger.warn("Unable to lock JVM memory (ENOMEM)."
+                        + " This can result in part of the JVM being swapped out, especially with mmapped I/O enabled."
+                        + " Increase RLIMIT_MEMLOCK or run Cassandra as root.");
+            }
+            else if (osType != MAC)
+            {
+                // OS X allows mlockall to be called, but always returns an error
+                logger.warn("Unknown mlockall error {}", errno(e));
+            }
+        }
+    }
+
+    public static void trySkipCache(String path, long offset, long len)
+    {
+        File f = new File(path);
+        if (!f.exists())
+            return;
+
+        try (FileInputStream fis = new FileInputStream(f))
+        {
+            trySkipCache(getfd(fis.getChannel()), offset, len, path);
+        }
+        catch (IOException e)
+        {
+            logger.warn("Could not skip cache", e);
+        }
+    }
+
+    public static void trySkipCache(int fd, long offset, long len, String path)
+    {
+        if (len == 0)
+            trySkipCache(fd, 0, 0, path);
+
+        while (len > 0)
+        {
+            int sublen = (int) Math.min(Integer.MAX_VALUE, len);
+            trySkipCache(fd, offset, sublen, path);
+            len -= sublen;
+            offset -= sublen;
+        }
+    }
+
+    public static void trySkipCache(int fd, long offset, int len, String path)
+    {
+        if (fd < 0)
+            return;
+
+        try
+        {
+            if (osType == LINUX)
+            {
+                int result = wrappedLibrary.callPosixFadvise(fd, offset, len, POSIX_FADV_DONTNEED);
+                if (result != 0)
+                    NoSpamLogger.log(
+                            logger,
+                            NoSpamLogger.Level.WARN,
+                            10,
+                            TimeUnit.MINUTES,
+                            "Failed trySkipCache on file: {} Error: " + wrappedLibrary.callStrerror(result).getString(0),
+                            path);
+            }
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // if JNA is unavailable just skipping Direct I/O
+            // instance of this class will act like normal RandomAccessFile
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            logger.warn(String.format("posix_fadvise(%d, %d) failed, errno (%d).", fd, offset, errno(e)));
+        }
+    }
+
+    public static int tryFcntl(int fd, int command, int flags)
+    {
+        // fcntl return value may or may not be useful, depending on the command
+        int result = -1;
+
+        try
+        {
+            result = wrappedLibrary.callFcntl(fd, command, flags);
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // if JNA is unavailable just skipping
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            logger.warn(String.format("fcntl(%d, %d, %d) failed, errno (%d).", fd, command, flags, errno(e)));
+        }
+
+        return result;
+    }
+
+    public static int tryOpenDirectory(String path)
+    {
+        int fd = -1;
+
+        try
+        {
+            return wrappedLibrary.callOpen(path, O_RDONLY);
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // JNA is unavailable just skipping Direct I/O
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            logger.warn(String.format("open(%s, O_RDONLY) failed, errno (%d).", path, errno(e)));
+        }
+
+        return fd;
+    }
+
+    public static void trySync(int fd)
+    {
+        if (fd == -1)
+            return;
+
+        try
+        {
+            wrappedLibrary.callFsync(fd);
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // JNA is unavailable just skipping Direct I/O
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            logger.warn("fsync({}) failed, errorno ({}) {}", fd, errno(e), e);
+        }
+    }
+
+    public static void tryCloseFD(int fd)
+    {
+        if (fd == -1)
+            return;
+
+        try
+        {
+            wrappedLibrary.callClose(fd);
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            // JNA is unavailable just skipping Direct I/O
+        }
+        catch (RuntimeException e)
+        {
+            if (!(e instanceof LastErrorException))
+                throw e;
+
+            logger.warn(String.format("close(%d) failed, errno (%d).", fd, errno(e)));
+        }
+    }
+
+    public static int getfd(FileChannel channel)
+    {
+        Field field = FBUtilities.getProtectedField(channel.getClass(), "fd");
+
+        try
+        {
+            return getfd((FileDescriptor)field.get(channel));
+        }
+        catch (IllegalArgumentException|IllegalAccessException e)
+        {
+            logger.warn("Unable to read fd field from FileChannel");
+        }
+        return -1;
+    }
+
+    /**
+     * Get system file descriptor from FileDescriptor object.
+     * @param descriptor - FileDescriptor objec to get fd from
+     * @return file descriptor, -1 or error
+     */
+    public static int getfd(FileDescriptor descriptor)
+    {
+        Field field = FBUtilities.getProtectedField(descriptor.getClass(), "fd");
+
+        try
+        {
+            return field.getInt(descriptor);
+        }
+        catch (Exception e)
+        {
+            JVMStabilityInspector.inspectThrowable(e);
+            logger.warn("Unable to read fd field from FileDescriptor");
+        }
+
+        return -1;
+    }
+
+    /**
+     * @return the PID of the JVM or -1 if we failed to get the PID
+     */
+    public static long getProcessID()
+    {
+        try
+        {
+            return wrappedLibrary.callGetpid();
+        }
+        catch (Exception e)
+        {
+            logger.info("Failed to get PID from JNA", e);
+        }
+
+        return -1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/NativeLibraryDarwin.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibraryDarwin.java b/src/java/org/apache/cassandra/utils/NativeLibraryDarwin.java
new file mode 100644
index 0000000..d6f1a9e
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeLibraryDarwin.java
@@ -0,0 +1,129 @@
+/*
+ * 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.cassandra.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jna.LastErrorException;
+import com.sun.jna.Native;
+import com.sun.jna.Pointer;
+
+/**
+ * A {@code NativeLibraryWrapper} implementation for Darwin/Mac.
+ * <p>
+ * When JNA is initialized, all methods that have the 'native' keyword
+ * will be attmpted to be linked against. As Java doesn't have the equivalent
+ * of a #ifdef, this means if a native method like posix_fadvise is defined in the
+ * class but not available on the target operating system (e.g.
+ * posix_fadvise is not availble on Darwin/Mac) this will cause the entire
+ * initial linking and initialization of JNA to fail. This means other
+ * native calls that are supported on that target operating system will be
+ * unavailable simply because of one native defined method not supported
+ * on the runtime operating system.
+ * @see org.apache.cassandra.utils.NativeLibraryWrapper
+ * @see NativeLibrary
+ */
+public class NativeLibraryDarwin implements NativeLibraryWrapper
+{
+    private static final Logger logger = LoggerFactory.getLogger(NativeLibraryDarwin.class);
+
+    private static boolean available;
+
+    static
+    {
+        try
+        {
+            Native.register("c");
+            available = true;
+        }
+        catch (NoClassDefFoundError e)
+        {
+            logger.warn("JNA not found. Native methods will be disabled.");
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            logger.error("Failed to link the C library against JNA. Native methods will be unavailable.", e);
+        }
+        catch (NoSuchMethodError e)
+        {
+            logger.warn("Obsolete version of JNA present; unable to register C library. Upgrade to JNA 3.2.7 or later");
+        }
+    }
+
+    private static native int mlockall(int flags) throws LastErrorException;
+    private static native int munlockall() throws LastErrorException;
+    private static native int fcntl(int fd, int command, long flags) throws LastErrorException;
+    private static native int open(String path, int flags) throws LastErrorException;
+    private static native int fsync(int fd) throws LastErrorException;
+    private static native int close(int fd) throws LastErrorException;
+    private static native Pointer strerror(int errnum) throws LastErrorException;
+    private static native long getpid() throws LastErrorException;
+
+    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return mlockall(flags);
+    }
+
+    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
+    {
+        return munlockall();
+    }
+
+    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return fcntl(fd, command, flags);
+    }
+
+    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
+    {
+        // posix_fadvise is not available on Darwin/Mac
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return open(path, flags);
+    }
+
+    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return fsync(fd);
+    }
+
+    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return close(fd);
+    }
+
+    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return strerror(errnum);
+    }
+
+    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
+    {
+        return getpid();
+    }
+
+    public boolean isAvailable()
+    {
+        return available;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/NativeLibraryLinux.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibraryLinux.java b/src/java/org/apache/cassandra/utils/NativeLibraryLinux.java
new file mode 100644
index 0000000..b6667e4
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeLibraryLinux.java
@@ -0,0 +1,129 @@
+/*
+ * 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.cassandra.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jna.LastErrorException;
+import com.sun.jna.Native;
+import com.sun.jna.Pointer;
+
+/**
+ * A {@code NativeLibraryWrapper} implementation for Linux.
+ * <p>
+ * When JNA is initialized, all methods that have the 'native' keyword
+ * will be attmpted to be linked against. As Java doesn't have the equivalent
+ * of a #ifdef, this means if a native method like posix_fadvise is defined in the
+ * class but not available on the target operating system (e.g.
+ * posix_fadvise is not availble on Darwin/Mac) this will cause the entire
+ * initial linking and initialization of JNA to fail. This means other
+ * native calls that are supported on that target operating system will be
+ * unavailable simply because of one native defined method not supported
+ * on the runtime operating system.
+ * @see org.apache.cassandra.utils.NativeLibraryWrapper
+ * @see NativeLibrary
+ */
+public class NativeLibraryLinux implements NativeLibraryWrapper
+{
+    private static boolean available;
+
+    private static final Logger logger = LoggerFactory.getLogger(NativeLibraryLinux.class);
+
+    static
+    {
+        try
+        {
+            Native.register("c");
+            available = true;
+        }
+        catch (NoClassDefFoundError e)
+        {
+            logger.warn("JNA not found. Native methods will be disabled.");
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            logger.error("Failed to link the C library against JNA. Native methods will be unavailable.", e);
+        }
+        catch (NoSuchMethodError e)
+        {
+            logger.warn("Obsolete version of JNA present; unable to register C library. Upgrade to JNA 3.2.7 or later");
+        }
+    }
+
+    private static native int mlockall(int flags) throws LastErrorException;
+    private static native int munlockall() throws LastErrorException;
+    private static native int fcntl(int fd, int command, long flags) throws LastErrorException;
+    private static native int posix_fadvise(int fd, long offset, int len, int flag) throws LastErrorException;
+    private static native int open(String path, int flags) throws LastErrorException;
+    private static native int fsync(int fd) throws LastErrorException;
+    private static native int close(int fd) throws LastErrorException;
+    private static native Pointer strerror(int errnum) throws LastErrorException;
+    private static native long getpid() throws LastErrorException;
+
+    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return mlockall(flags);
+    }
+
+    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
+    {
+        return munlockall();
+    }
+
+    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return fcntl(fd, command, flags);
+    }
+
+    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return posix_fadvise(fd, offset, len, flag);
+    }
+
+    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return open(path, flags);
+    }
+
+    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return fsync(fd);
+    }
+
+    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return close(fd);
+    }
+
+    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
+    {
+        return strerror(errnum);
+    }
+
+    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
+    {
+        return getpid();
+    }
+
+    public boolean isAvailable()
+    {
+        return available;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/NativeLibraryWindows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibraryWindows.java b/src/java/org/apache/cassandra/utils/NativeLibraryWindows.java
new file mode 100644
index 0000000..0868c7a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeLibraryWindows.java
@@ -0,0 +1,124 @@
+/*
+ * 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.cassandra.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jna.LastErrorException;
+import com.sun.jna.Native;
+import com.sun.jna.Pointer;
+
+/**
+ * A {@code NativeLibraryWrapper} implementation for Windows.
+ * <p> This implementation only offer support for the {@code callGetpid} method
+ * using the Windows/Kernel32 library.</p>
+ *
+ * @see org.apache.cassandra.utils.NativeLibraryWrapper
+ * @see NativeLibrary
+ */
+public class NativeLibraryWindows implements NativeLibraryWrapper
+{
+    private static boolean available;
+
+    private static final Logger logger = LoggerFactory.getLogger(NativeLibraryWindows.class);
+
+    static
+    {
+        try
+        {
+            Native.register("kernel32");
+            available = true;
+        }
+        catch (NoClassDefFoundError e)
+        {
+            logger.warn("JNA not found. Native methods will be disabled.");
+        }
+        catch (UnsatisfiedLinkError e)
+        {
+            logger.error("Failed to link the Windows/Kernel32 library against JNA. Native methods will be unavailable.", e);
+        }
+        catch (NoSuchMethodError e)
+        {
+            logger.warn("Obsolete version of JNA present; unable to register Windows/Kernel32 library. Upgrade to JNA 3.2.7 or later");
+        }
+    }
+
+    /**
+     * Retrieves the process identifier of the calling process (<a href='https://msdn.microsoft.com/en-us/library/windows/desktop/ms683180(v=vs.85).aspx'>GetCurrentProcessId function</a>).
+     *
+     * @return the process identifier of the calling process
+     */
+    private static native long GetCurrentProcessId() throws LastErrorException;
+
+    public int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callMunlockall() throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    public Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException
+    {
+        throw new UnsatisfiedLinkError();
+    }
+
+    /**
+     * @return the PID of the JVM running
+     * @throws UnsatisfiedLinkError if we fail to link against Sigar
+     * @throws RuntimeException if another unexpected error is thrown by Sigar
+     */
+    public long callGetpid() throws UnsatisfiedLinkError, RuntimeException
+    {
+        return GetCurrentProcessId();
+    }
+
+    public boolean isAvailable()
+    {
+        return available;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/NativeLibraryWrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeLibraryWrapper.java b/src/java/org/apache/cassandra/utils/NativeLibraryWrapper.java
new file mode 100644
index 0000000..879ea88
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeLibraryWrapper.java
@@ -0,0 +1,44 @@
+/*
+ * 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.cassandra.utils;
+
+import com.sun.jna.Pointer;
+
+/**
+ * An interface to implement for using OS specific native methods.
+ * @see NativeLibrary
+ */
+interface NativeLibraryWrapper
+{
+    /**
+     * Checks if the library has been successfully linked.
+     * @return {@code true} if the library has been successfully linked, {@code false} otherwise.
+     */
+    boolean isAvailable();
+
+    int callMlockall(int flags) throws UnsatisfiedLinkError, RuntimeException;
+    int callMunlockall() throws UnsatisfiedLinkError, RuntimeException;
+    int callFcntl(int fd, int command, long flags) throws UnsatisfiedLinkError, RuntimeException;
+    int callPosixFadvise(int fd, long offset, int len, int flag) throws UnsatisfiedLinkError, RuntimeException;
+    int callOpen(String path, int flags) throws UnsatisfiedLinkError, RuntimeException;
+    int callFsync(int fd) throws UnsatisfiedLinkError, RuntimeException;
+    int callClose(int fd) throws UnsatisfiedLinkError, RuntimeException;
+    Pointer callStrerror(int errnum) throws UnsatisfiedLinkError, RuntimeException;
+    long callGetpid() throws UnsatisfiedLinkError, RuntimeException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/SyncUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SyncUtil.java b/src/java/org/apache/cassandra/utils/SyncUtil.java
index 4c0d89d..64d64cf 100644
--- a/src/java/org/apache/cassandra/utils/SyncUtil.java
+++ b/src/java/org/apache/cassandra/utils/SyncUtil.java
@@ -176,7 +176,7 @@ public class SyncUtil
         if (SKIP_SYNC)
             return;
         else
-            CLibrary.trySync(fd);
+            NativeLibrary.trySync(fd);
     }
 
     public static void trySyncDir(File dir)
@@ -184,14 +184,14 @@ public class SyncUtil
         if (SKIP_SYNC)
             return;
 
-        int directoryFD = CLibrary.tryOpenDirectory(dir.getPath());
+        int directoryFD = NativeLibrary.tryOpenDirectory(dir.getPath());
         try
         {
             trySync(directoryFD);
         }
         finally
         {
-            CLibrary.tryCloseFD(directoryFD);
+            NativeLibrary.tryCloseFD(directoryFD);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 66649c2..11c1895 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -360,7 +360,7 @@ public class UUIDGen
                 messageDigest.update(addr.getAddress());
 
             // Identify the process on the load: we use both the PID and class loader hash.
-            long pid = CLibrary.getProcessID();
+            long pid = NativeLibrary.getProcessID();
             if (pid < 0)
                 pid = new Random(System.currentTimeMillis()).nextLong();
             FBUtilities.updateWithLong(messageDigest, pid);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/src/java/org/apache/cassandra/utils/WindowsTimer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/WindowsTimer.java b/src/java/org/apache/cassandra/utils/WindowsTimer.java
index 9db8559..351751f 100644
--- a/src/java/org/apache/cassandra/utils/WindowsTimer.java
+++ b/src/java/org/apache/cassandra/utils/WindowsTimer.java
@@ -34,6 +34,10 @@ public final class WindowsTimer
         {
             Native.register("winmm");
         }
+        catch (NoClassDefFoundError e)
+        {
+            logger.warn("JNA not found. winmm.dll cannot be registered. Performance will be negatively impacted on this node.");
+        }
         catch (Exception e)
         {
             logger.error("Failed to register winmm.dll. Performance will be negatively impacted on this node.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/test/unit/org/apache/cassandra/utils/CLibraryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/CLibraryTest.java b/test/unit/org/apache/cassandra/utils/CLibraryTest.java
deleted file mode 100644
index d07b1ff..0000000
--- a/test/unit/org/apache/cassandra/utils/CLibraryTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.cassandra.utils;
-
-import java.io.File;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.io.util.FileUtils;
-
-public class CLibraryTest
-{
-    @Test
-    public void testSkipCache()
-    {
-        File file = FileUtils.createTempFile("testSkipCache", "1");
-
-        CLibrary.trySkipCache(file.getPath(), 0, 0);
-    }
-
-    @Test
-    public void getPid()
-    {
-        long pid = CLibrary.getProcessID();
-        Assert.assertTrue(pid > 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8692c6/test/unit/org/apache/cassandra/utils/NativeLibraryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/NativeLibraryTest.java b/test/unit/org/apache/cassandra/utils/NativeLibraryTest.java
new file mode 100644
index 0000000..226653e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/NativeLibraryTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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.cassandra.utils;
+
+import java.io.File;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.FileUtils;
+
+public class NativeLibraryTest
+{
+    @Test
+    public void testSkipCache()
+    {
+        File file = FileUtils.createTempFile("testSkipCache", "1");
+
+        NativeLibrary.trySkipCache(file.getPath(), 0, 0);
+    }
+
+    @Test
+    public void getPid()
+    {
+        long pid = NativeLibrary.getProcessID();
+        Assert.assertTrue(pid > 0);
+    }
+}