You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by mm...@apache.org on 2018/11/13 18:35:30 UTC

[bookkeeper] branch master updated: Added module to enable CPU affinity

This is an automated email from the ASF dual-hosted git repository.

mmerli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new dab8310  Added module to enable CPU affinity
dab8310 is described below

commit dab8310f09c963112bb7a3efede5763ac35ee185
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Tue Nov 13 10:35:24 2018 -0800

    Added module to enable CPU affinity
    
    ### Motivation
    
    This is part of a set of changes aimed at reducing latency in BK at the expense of other aspects (eg: max throughput). While not intended to be used as default settings, they might be good to have whenever the latency becomes critical.
    
    Pinning a thread to a particular CPU will ensure no other process will execute on that CPU reducing all scheduler induced context switches that will cause latency jittery.
    
    A given thread that wants to get pinned to a CPU just needs to call:
    
    ```java
    CpuAffinity.acquireCore();
    ```
    
    It's called `acquireCore()` because it will also disable hyper-threading on the pinned cpu.
    
    Subsequent PRs will use this module to have the option to pin critical threads to available CPUs.
    
    ### Changes
     * Added JNI module to call `sched_setaffinity()` to pin a thread to a particular CPU
     * Automatically discover available isolated CPUs
     * Acquire file-based locks to allow multiple processes on same machine to acquire CPUs independently.
    
    Reviewers: Ivan Kelly <iv...@apache.org>, Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1641 from merlimat/cpu-affinity
---
 .../com/scurrilous/circe/utils/NativeUtils.java    |  15 +-
 cpu-affinity/pom.xml                               | 181 ++++++
 cpu-affinity/src/main/affinity/cpp/affinity_jni.c  |  96 +++
 cpu-affinity/src/main/assembly/assembly.xml        |  62 ++
 .../common/util/affinity/CpuAffinity.java          |  51 ++
 .../common/util/affinity/impl/CpuAffinityImpl.java | 218 +++++++
 .../common/util/affinity/impl/CpuAffinityJni.java  |  37 ++
 .../util/affinity/impl/IsolatedProcessors.java     | 102 ++++
 .../common/util/affinity/impl}/NativeUtils.java    |  56 +-
 .../common/util/affinity/impl/ProcessorsInfo.java  |  99 ++++
 .../common/util/affinity/impl/package-info.java    |  22 +
 .../common/util/affinity/package-info.java         |  22 +
 .../common/util/affinity/impl/CpuInfoTest.java     |  53 ++
 .../util/affinity/impl/IsolatedProcessorsTest.java |  47 ++
 cpu-affinity/src/test/resources/proc_cpuinfo.txt   | 648 +++++++++++++++++++++
 pom.xml                                            |   3 +-
 16 files changed, 1668 insertions(+), 44 deletions(-)

diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java b/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java
index 8776092..05ace29 100644
--- a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java
@@ -37,7 +37,7 @@ public class NativeUtils {
 
     /**
      * loads given library from the this jar. ie: this jar contains: /lib/pulsar-checksum.jnilib
-     * 
+     *
      * @param path
      *            : absolute path of the library in the jar <br/>
      *            if this jar contains: /lib/pulsar-checksum.jnilib then provide the same absolute path as input
@@ -67,13 +67,16 @@ public class NativeUtils {
             throw new FileNotFoundException("Couldn't find file into jar " + path);
         }
 
-        OutputStream out = new FileOutputStream(temp);
         try {
-            while ((read = input.read(buffer)) != -1) {
-                out.write(buffer, 0, read);
+            OutputStream out = new FileOutputStream(temp);
+            try {
+                while ((read = input.read(buffer)) != -1) {
+                    out.write(buffer, 0, read);
+                }
+            } finally {
+                out.close();
             }
         } finally {
-            out.close();
             input.close();
         }
 
@@ -87,7 +90,7 @@ public class NativeUtils {
     /**
      * Returns jni library extension based on OS specification. Maven-nar generates jni library based on different OS :
      * http://mark.donszelmann.org/maven-nar-plugin/aol.html (jni.extension)
-     * 
+     *
      * @return
      */
     public static String libType() {
diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml
new file mode 100644
index 0000000..f58a079
--- /dev/null
+++ b/cpu-affinity/pom.xml
@@ -0,0 +1,181 @@
+<!--
+   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.
+-->
+<project
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper</groupId>
+    <artifactId>bookkeeper</artifactId>
+    <version>4.9.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>cpu-affinity</artifactId>
+  <packaging>nar</packaging>
+  <name>Apache BookKeeper :: CPU Affinity Library</name>
+  <description>CPU Affinity Library</description>
+
+  <properties>
+    <nar.runtime>dynamic</nar.runtime>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+          <compilerArgs>
+            <!-- Object.finalize() is deprecated at java 9 -->
+            <!-- <compilerArg>-Werror</compilerArg> -->
+            <compilerArg>-Xlint:deprecation</compilerArg>
+            <compilerArg>-Xlint:unchecked</compilerArg>
+            <!-- https://issues.apache.org/jira/browse/MCOMPILER-205 -->
+            <compilerArg>-Xpkginfo:always</compilerArg>
+          </compilerArgs>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>com.github.maven-nar</groupId>
+        <artifactId>nar-maven-plugin</artifactId>
+        <version>${nar-maven-plugin.version}</version>
+        <extensions>true</extensions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <descriptors>
+            <descriptor>src/main/assembly/assembly.xml</descriptor>
+          </descriptors>
+          <appendAssemblyId>false</appendAssemblyId>
+          <tarLongFileMode>posix</tarLongFileMode>
+        </configuration>
+        <executions>
+          <execution>
+            <id>make-assembly</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <!-- exclude test files -->
+            <exclude>**/src/test/resources/proc_cpuinfo.txt</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>mac</id>
+      <activation>
+        <os>
+          <name>Mac OS X</name>
+        </os>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>com.github.maven-nar</groupId>
+            <artifactId>nar-maven-plugin</artifactId>
+            <version>${nar-maven-plugin.version}</version>
+            <extensions>true</extensions>
+            <configuration>
+              <runtime>${nar.runtime}</runtime>
+              <output>cpu-affinity</output>
+              <libraries>
+                <library>
+                  <type>jni</type>
+                  <narSystemPackage>org.apache.bookkeeper.utils.affinity</narSystemPackage>
+                </library>
+              </libraries>
+              <cpp>
+                <optionSet>${nar.cpp.optionSet}</optionSet>
+                <exceptions>false</exceptions>
+                <rtti>false</rtti>
+                <optimize>full</optimize>
+              </cpp>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <profile>
+      <id>Linux</id>
+      <activation>
+        <os>
+          <name>Linux</name>
+        </os>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>com.github.maven-nar</groupId>
+            <artifactId>nar-maven-plugin</artifactId>
+            <version>${nar-maven-plugin.version}</version>
+            <extensions>true</extensions>
+            <configuration>
+              <runtime>${nar.runtime}</runtime>
+              <output>cpu-affinity</output>
+              <libraries>
+                <library>
+                  <type>jni</type>
+                  <narSystemPackage>org.apache.bookkeeper.utils.affinity</narSystemPackage>
+                </library>
+              </libraries>
+              <cpp>
+                <optionSet>${nar.cpp.optionSet}</optionSet>
+                <exceptions>false</exceptions>
+                <rtti>false</rtti>
+                <optimize>full</optimize>
+              </cpp>
+              <linker>
+                <libSet>rt</libSet>
+              </linker>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+</project>
diff --git a/cpu-affinity/src/main/affinity/cpp/affinity_jni.c b/cpu-affinity/src/main/affinity/cpp/affinity_jni.c
new file mode 100644
index 0000000..a0aedf6
--- /dev/null
+++ b/cpu-affinity/src/main/affinity/cpp/affinity_jni.c
@@ -0,0 +1,96 @@
+/**
+ *
+ * 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.
+ *
+ */
+
+// Use different error code to differentiate non-implemented error
+static const int NOT_IMPLEMENTED = -2;
+
+#ifdef __linux__
+#define _GNU_SOURCE
+#include <sched.h>
+#include <unistd.h>
+#include <sys/syscall.h>
+
+static int set_affinity(int cpuid) {
+    cpu_set_t cpus;
+    CPU_ZERO(&cpus);
+    CPU_SET((size_t)cpuid, &cpus);
+    int threadId = (int)syscall(SYS_gettid);
+    return sched_setaffinity(threadId, sizeof(cpu_set_t), &cpus);
+}
+
+static const int IS_AVAILABLE = 1;
+
+#else
+
+static int set_affinity(int cpuid) { return NOT_IMPLEMENTED; }
+
+static const int IS_AVAILABLE = 0;
+
+#endif
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <errno.h>
+#include <string.h>
+
+#include <org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni.h>
+
+/*
+ * Class:     org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni
+ * Method:    isRoot
+ * Signature: ()Z
+ */
+JNIEXPORT jboolean JNICALL
+Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_isRoot(JNIEnv *env, jclass cls) {
+    return getuid() == 0;
+}
+
+/*
+ * Class:     org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni
+ * Method:    isAvailable
+ * Signature: ()Z
+ */
+JNIEXPORT jboolean JNICALL
+Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_isAvailable(JNIEnv *env, jclass cls) {
+    return IS_AVAILABLE == 1;
+}
+
+/*
+ * Class:     org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni
+ * Method:    setAffinity
+ * Signature: (I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_setAffinity(
+    JNIEnv *env, jclass cls, jint cpuid) {
+    int res = set_affinity(cpuid);
+
+    if (res == 0) {
+        // Success
+        return;
+    } else if (res == NOT_IMPLEMENTED) {
+        (*env)->ThrowNew(env, (*env)->FindClass(env, "java/lang/Exception"), "CPU affinity not implemented");
+    } else {
+        // Error in sched_setaffinity, get message from errno
+        char buffer[1024];
+        strerror_r(errno, buffer, sizeof(buffer));
+        (*env)->ThrowNew(env, (*env)->FindClass(env, "java/lang/Exception"), buffer);
+    }
+}
diff --git a/cpu-affinity/src/main/assembly/assembly.xml b/cpu-affinity/src/main/assembly/assembly.xml
new file mode 100644
index 0000000..377b97f
--- /dev/null
+++ b/cpu-affinity/src/main/assembly/assembly.xml
@@ -0,0 +1,62 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    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.
+
+-->
+<assembly>
+	<id>all</id>
+	<formats>
+		<format>jar</format>
+	</formats>
+
+	<includeBaseDirectory>false</includeBaseDirectory>
+  	<fileSets>
+		<fileSet>
+			<directory>${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-MacOSX-gpp-jni/lib/${os.arch}-MacOSX-gpp/jni
+			</directory>
+			<outputDirectory>lib</outputDirectory>
+			<includes>
+				<include>lib*</include>
+			</includes>
+		</fileSet>
+		<fileSet>
+			<directory>${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-Linux-gpp-jni/lib/${os.arch}-Linux-gpp/jni
+			</directory>
+			<outputDirectory>lib</outputDirectory>
+			<includes>
+				<include>lib*</include>
+			</includes>
+		</fileSet>
+		<fileSet>
+			<directory>${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-${os.name}-gpp-jni/lib/${os.arch}-${os.name}-gpp/jni
+			</directory>
+			<outputDirectory>lib</outputDirectory>
+			<includes>
+				<include>lib*</include>
+			</includes>
+		</fileSet>
+		<fileSet>
+			<directory>${project.build.directory}/classes</directory>
+			<outputDirectory></outputDirectory>
+			<includes>
+				<include>**/*</include>
+			</includes>
+		</fileSet>
+	</fileSets>
+</assembly>
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java
new file mode 100644
index 0000000..d0dab94
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java
@@ -0,0 +1,51 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.common.util.affinity;
+
+import lombok.experimental.UtilityClass;
+
+import org.apache.bookkeeper.common.util.affinity.impl.CpuAffinityImpl;
+
+/**
+ * Utilities for enabling thread to CPU affinity.
+ */
+@UtilityClass
+public class CpuAffinity {
+
+    /**
+     * Acquire ownership of one CPU core for the current thread.
+     *
+     * <p>Notes:
+     *
+     * <ol>
+     * <li>This method will only consider CPUs that are "isolated" by the OS. Eg: boot the kernel with
+     * <code>isolcpus=2,3,6,7</code> parameter
+     * <li>
+     * <li>This method will disable hyper-threading on the owned core
+     * <li>Once a thread successfully acquires a CPU, ownership will be retained, even if the thread exits, for as long
+     * as the JVM process is alive.
+     * </ol>
+     */
+    public static void acquireCore() {
+        CpuAffinityImpl.acquireCore();
+    }
+}
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java
new file mode 100644
index 0000000..f3c80a9
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java
@@ -0,0 +1,218 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import lombok.experimental.UtilityClass;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * Implementation of CPU Affinity functionality.
+ */
+@UtilityClass
+@Slf4j
+public class CpuAffinityImpl {
+
+    private static boolean isInitialized = false;
+    private static boolean isSupported;
+
+    // Id of CPU cores acquired by this process
+    private static final SortedSet<Integer> acquiredProcessors = new TreeSet<>();
+
+    // Ids of processors that were isolated by Linux at boot time. This is the set
+    // of processors that can acquired by this or other processes.
+    private static SortedSet<Integer> isolatedProcessors = null;
+
+    private static ProcessorsInfo processorsInfo = null;
+
+    public static synchronized void acquireCore() {
+        if (!isInitialized) {
+            init();
+        }
+
+        if (!isSupported) {
+            throw new RuntimeException("CPU Affinity not supported in current environment");
+        }
+
+        if (!CpuAffinityJni.isRoot()) {
+            throw new RuntimeException("CPU Affinity can only be set if the process is running as root");
+        }
+
+        try {
+            int cpu = pickAvailableCpu();
+            CpuAffinityJni.setAffinity(cpu);
+
+            log.info("Thread {} has successfully acquired ownership of cpu {}", Thread.currentThread().getName(), cpu);
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to acquire CPU core: " + e.getMessage());
+        }
+    }
+
+    private static final String LOCK_FILE_PREFIX = Paths.get(System.getProperty("java.io.tmpdir"), "cpu-lock-")
+            .toString();
+
+    /**
+     * Other than the cores acquired by this process, there might be other processes on the same host trying to acquire
+     * the available cores.
+     *
+     * <p>We use file-locks to ensure that other processes are aware of which CPUs are taken and that these locks are
+     * automatically released if the process crashes.
+     */
+    private static synchronized int pickAvailableCpu() throws IOException {
+        if (isolatedProcessors == null) {
+            isolatedProcessors = IsolatedProcessors.get();
+        }
+        for (int isolatedCpu : isolatedProcessors) {
+            if (log.isDebugEnabled()) {
+                log.debug("Checking CPU {}", isolatedCpu);
+            }
+            if (acquiredProcessors.contains(isolatedCpu)) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Ignoring CPU {} since it's already acquired", isolatedCpu);
+                }
+                continue;
+            }
+
+            if (tryAcquireCpu(isolatedCpu)) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Using CPU {}", isolatedCpu);
+                }
+                return isolatedCpu;
+            }
+        }
+
+        throw new RuntimeException(
+                "There is no available isolated CPU to acquire for thread " + Thread.currentThread().getName());
+    }
+
+    private static boolean tryAcquireCpu(int targetCpu) throws IOException {
+        // First, acquire lock on all the cpus that share the same core as target cpu
+        if (processorsInfo == null) {
+            processorsInfo = ProcessorsInfo.parseCpuInfo();
+        }
+
+        Set<Integer> cpusToAcquire = processorsInfo.getCpusOnSameCore(targetCpu);
+        List<Closeable> acquiredCpus = new ArrayList<>();
+
+        for (int cpu : cpusToAcquire) {
+            Closeable lock = tryAcquireFileLock(cpu);
+            if (lock == null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Failed to acquire lock on CPU {}", cpu);
+                }
+
+                // Failed to acquire one cpu, release the rest that were already locked
+                for (Closeable l : acquiredCpus) {
+                    l.close();
+                }
+
+                return false;
+            } else {
+                acquiredCpus.add(lock);
+            }
+        }
+
+        // At this point, we have ownership of all required cpus
+        // Make sure the requested CPU is enabled and that all other CPUs on the same core are disabled, so that
+        // hyper-threading will not affect latency.
+        for (int cpu : cpusToAcquire) {
+            if (cpu == targetCpu) {
+                IsolatedProcessors.enableCpu(cpu);
+            } else {
+                IsolatedProcessors.disableCpu(cpu);
+            }
+
+            acquiredProcessors.add(cpu);
+        }
+        return true;
+    }
+
+    /**
+     * Try to acquire a lock on a particular cpu.
+     *
+     * @return null if the lock was not available
+     * @return a {@link Closeable} lock object if the lock was acquired
+     */
+    private static Closeable tryAcquireFileLock(int cpu) throws IOException {
+        String lockPath = LOCK_FILE_PREFIX + cpu;
+
+        RandomAccessFile file = null;
+        FileChannel channel = null;
+        FileLock lock = null;
+
+        try {
+            file = new RandomAccessFile(new File(lockPath), "rw");
+            channel = file.getChannel();
+            lock = channel.tryLock();
+
+            if (lock == null) {
+                return null;
+            } else {
+                final FileLock finalLock = lock;
+                final FileChannel finalChannel = channel;
+                final RandomAccessFile finalFile = file;
+                return () -> {
+                    // Closable object
+                    finalLock.close();
+                    finalChannel.close();
+                    finalFile.close();
+                };
+            }
+        } finally {
+            if (lock == null) {
+                if (channel != null) {
+                    channel.close();
+                }
+
+                if (file != null) {
+                    file.close();
+                }
+            }
+        }
+    }
+
+    private static void init() {
+        try {
+            // Since this feature is only available in Linux, there's no point
+            // in checking for MacOS jnilib or Windows dll extensions
+            NativeUtils.loadLibraryFromJar("/lib/libcpu-affinity.so");
+            isSupported = CpuAffinityJni.isAvailable();
+        } catch (final Exception | UnsatisfiedLinkError e) {
+            log.warn("Unable to load CPU affinity library: {}", e.getMessage(), e);
+            isSupported = false;
+        } finally {
+            isInitialized = true;
+        }
+    }
+
+}
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java
new file mode 100644
index 0000000..30ee325
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java
@@ -0,0 +1,37 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import java.io.IOException;
+
+import lombok.experimental.UtilityClass;
+
+/**
+ * JNI wrapper of native functions for CPU affinity.
+ */
+@UtilityClass
+public class CpuAffinityJni {
+    static native boolean isRoot();
+
+    static native boolean isAvailable();
+
+    static native void setAffinity(int cpuId) throws IOException;
+}
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java
new file mode 100644
index 0000000..fd9cbbc
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java
@@ -0,0 +1,102 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import com.google.common.base.Charsets;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import lombok.experimental.UtilityClass;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.commons.lang3.StringUtils;
+
+@UtilityClass
+@Slf4j
+class IsolatedProcessors {
+
+    private static final Charset ENCODING = Charsets.US_ASCII;
+
+    private static final String ISOLATED_CPU_PATH = "/sys/devices/system/cpu/isolated";
+
+    @SuppressFBWarnings("DMI_HARDCODED_ABSOLUTE_FILENAME")
+    static SortedSet<Integer> get() throws IOException {
+        return parseProcessorRange(new String(Files.readAllBytes(Paths.get(ISOLATED_CPU_PATH)), ENCODING));
+    }
+
+    static SortedSet<Integer> parseProcessorRange(String range) {
+        SortedSet<Integer> processors = new TreeSet<>();
+
+        for (String part : StringUtils.trim(range).split(",")) {
+            if (part.contains("-")) {
+                // This is a range, eg: 1-5 with both edges included
+                String[] parts = part.split("-");
+                int first = Integer.parseInt(parts[0]);
+                int last = Integer.parseInt(parts[1]);
+
+                for (int i = first; i <= last; i++) {
+                    processors.add(i);
+                }
+            } else if (!part.isEmpty()) {
+                processors.add(Integer.parseInt(part));
+            }
+        }
+
+        return processors;
+    }
+
+    /**
+     * Instruct Linux to disable a particular CPU. This is used to disable hyper-threading on a particular core, by
+     * shutting down the cpu that shares the same core.
+     */
+    static void disableCpu(int cpu) throws IOException {
+        changeCpuStatus(cpu, false);
+    }
+
+    static void enableCpu(int cpu) throws IOException {
+        changeCpuStatus(cpu, true);
+    }
+
+    /**
+     * Instruct Linux to disable a particular CPU. This is used to disable hyper-threading on a particular core, by
+     * shutting down the cpu that shares the same core.
+     */
+    private static void changeCpuStatus(int cpu, boolean enable) throws IOException {
+        Path cpuPath = Paths.get(String.format("/sys/devices/system/cpu/cpu%d/online", cpu));
+
+        boolean currentState = Integer
+                .parseInt(StringUtils.trim(new String(Files.readAllBytes(cpuPath), ENCODING))) != 0;
+
+        if (currentState != enable) {
+            Files.write(cpuPath, (enable ? "1\n" : "0\n").getBytes(ENCODING), StandardOpenOption.TRUNCATE_EXISTING);
+            log.info("{} CPU {}", enable ? "Enabled" : "Disabled", cpu);
+        }
+    }
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java
similarity index 65%
copy from circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java
copy to cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java
index 8776092..67c8679 100644
--- a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java
@@ -1,4 +1,5 @@
 /**
+ *
  * 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
@@ -15,43 +16,46 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package com.scurrilous.circe.utils;
+package org.apache.bookkeeper.common.util.affinity.impl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.Locale;
+
+import lombok.experimental.UtilityClass;
 
 /**
- * Utils for loading native checksum library.
+ * Utility class to load jni library from inside a JAR.
  */
+@UtilityClass
 public class NativeUtils {
-
-    public static final String OS_NAME = System.getProperty("os.name").toLowerCase(Locale.US);
-
     /**
      * loads given library from the this jar. ie: this jar contains: /lib/pulsar-checksum.jnilib
-     * 
+     *
      * @param path
      *            : absolute path of the library in the jar <br/>
      *            if this jar contains: /lib/pulsar-checksum.jnilib then provide the same absolute path as input
      * @throws Exception
      */
+    @SuppressFBWarnings(
+            value = "OBL_UNSATISFIED_OBLIGATION",
+            justification = "work around for java 9: https://github.com/spotbugs/spotbugs/issues/493")
     public static void loadLibraryFromJar(String path) throws Exception {
-
-        checkArgument(path.startsWith("/"), "absolute path must start with  /");
+        checkArgument(path.startsWith("/"), "absolute path must start with /");
 
         String[] parts = path.split("/");
         String filename = (parts.length > 0) ? parts[parts.length - 1] : null;
 
         File dir = File.createTempFile("native", "");
-        dir.delete();
         if (!(dir.mkdir())) {
             throw new IOException("Failed to create temp directory " + dir.getAbsolutePath());
         }
@@ -62,19 +66,15 @@ public class NativeUtils {
         byte[] buffer = new byte[1024];
         int read;
 
-        InputStream input = NativeUtils.class.getResourceAsStream(path);
-        if (input == null) {
-            throw new FileNotFoundException("Couldn't find file into jar " + path);
-        }
+        try (InputStream input = NativeUtils.class.getResourceAsStream(path);
+                OutputStream out = new FileOutputStream(temp)) {
+            if (input == null) {
+                throw new FileNotFoundException("Couldn't find file into jar " + path);
+            }
 
-        OutputStream out = new FileOutputStream(temp);
-        try {
             while ((read = input.read(buffer)) != -1) {
                 out.write(buffer, 0, read);
             }
-        } finally {
-            out.close();
-            input.close();
         }
 
         if (!temp.exists()) {
@@ -83,22 +83,4 @@ public class NativeUtils {
 
         System.load(temp.getAbsolutePath());
     }
-
-    /**
-     * Returns jni library extension based on OS specification. Maven-nar generates jni library based on different OS :
-     * http://mark.donszelmann.org/maven-nar-plugin/aol.html (jni.extension)
-     * 
-     * @return
-     */
-    public static String libType() {
-
-        if (OS_NAME.indexOf("mac") >= 0) {
-            return "jnilib";
-        } else if (OS_NAME.indexOf("nix") >= 0 || OS_NAME.indexOf("nux") >= 0 || OS_NAME.indexOf("aix") > 0) {
-            return "so";
-        } else if (OS_NAME.indexOf("win") >= 0) {
-            return "dll";
-        }
-        throw new TypeNotPresentException(OS_NAME + " not supported", null);
-    }
 }
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java
new file mode 100644
index 0000000..5f4449b
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java
@@ -0,0 +1,99 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Charsets;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Discover the list of processors from /proc/cpuinfo.
+ */
+class ProcessorsInfo {
+
+    private static final Charset ENCODING = Charsets.US_ASCII;
+
+    /**
+     * Given one cpu id, return all CPUs that are sharing the same core.
+     */
+    public Set<Integer> getCpusOnSameCore(int targetCpu) {
+        Set<Integer> result = new TreeSet<>();
+        int targetCore = cpus.get(targetCpu);
+
+        cpus.forEach((cpu, core) -> {
+            if (core == targetCore) {
+                result.add(cpu);
+            }
+        });
+
+        return result;
+    }
+
+    private final Map<Integer, Integer> cpus = new TreeMap<>();
+
+    static ProcessorsInfo parseCpuInfo(String cpuInfoString) {
+        ProcessorsInfo pi = new ProcessorsInfo();
+
+        for (String cpu : cpuInfoString.split("\n\n")) {
+            int cpuId = -1;
+            int coreId = -1;
+
+            for (String line : cpu.split("\n")) {
+                String[] parts = line.split(":", 2);
+                String key = StringUtils.trim(parts[0]);
+                String value = StringUtils.trim(parts[1]);
+
+                if (key.equals("core id")) {
+                    coreId = Integer.parseInt(value);
+                } else if (key.equals("processor")) {
+                    cpuId = Integer.parseInt(value);
+                } else {
+                    // ignore
+                }
+            }
+
+            checkArgument(cpuId >= 0);
+            checkArgument(coreId >= 0);
+            pi.cpus.put(cpuId, coreId);
+        }
+
+        return pi;
+    }
+
+    static ProcessorsInfo parseCpuInfo() {
+        try {
+            return parseCpuInfo(new String(Files.readAllBytes(Paths.get("/proc/cpuinfo")), ENCODING));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java
new file mode 100644
index 0000000..35da816
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation of util functions for CPU affinity.
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
\ No newline at end of file
diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java
new file mode 100644
index 0000000..af4f4b4
--- /dev/null
+++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Util functions for CPU affinity.
+ */
+package org.apache.bookkeeper.common.util.affinity;
\ No newline at end of file
diff --git a/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java
new file mode 100644
index 0000000..6f8db0f
--- /dev/null
+++ b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.Sets;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.util.stream.Collectors;
+
+import org.junit.Test;
+
+/**
+ * Tests for CpuInfo class.
+ */
+public class CpuInfoTest {
+
+    @Test
+    public void testParseCpuInfo() throws Exception {
+        try (BufferedReader r = new BufferedReader(
+                new InputStreamReader(CpuInfoTest.class.getResourceAsStream("/proc_cpuinfo.txt")))) {
+            String text = r.lines().collect(Collectors.joining("\n"));
+
+            ProcessorsInfo pi = ProcessorsInfo.parseCpuInfo(text);
+
+            assertEquals(Sets.newHashSet(0, 12), pi.getCpusOnSameCore(0));
+            assertEquals(Sets.newHashSet(0, 12), pi.getCpusOnSameCore(12));
+
+            assertEquals(Sets.newHashSet(8, 20), pi.getCpusOnSameCore(8));
+            assertEquals(Sets.newHashSet(8, 20), pi.getCpusOnSameCore(20));
+        }
+    }
+}
diff --git a/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java
new file mode 100644
index 0000000..5bb68e4
--- /dev/null
+++ b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java
@@ -0,0 +1,47 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.common.util.affinity.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.Sets;
+
+import org.junit.Test;
+
+/**
+ * Tests for {@link IsolatedProcessors}.
+ */
+public class IsolatedProcessorsTest {
+
+    @Test
+    public void testParseProcessors() throws Exception {
+        assertEquals(Sets.newHashSet(), IsolatedProcessors.parseProcessorRange(""));
+
+        assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7), IsolatedProcessors.parseProcessorRange("1,3-7"));
+
+        assertEquals(Sets.newHashSet(1), IsolatedProcessors.parseProcessorRange("1"));
+        assertEquals(Sets.newHashSet(1, 3), IsolatedProcessors.parseProcessorRange("1,3"));
+        assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7, 10, 11, 12, 13),
+                IsolatedProcessors.parseProcessorRange("1,3-7,10-13"));
+
+        assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7), IsolatedProcessors.parseProcessorRange("1,3-7\n"));
+    }
+}
diff --git a/cpu-affinity/src/test/resources/proc_cpuinfo.txt b/cpu-affinity/src/test/resources/proc_cpuinfo.txt
new file mode 100644
index 0000000..9ffe3e3
--- /dev/null
+++ b/cpu-affinity/src/test/resources/proc_cpuinfo.txt
@@ -0,0 +1,648 @@
+processor	: 0
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 0
+cpu cores	: 12
+apicid		: 0
+initial apicid	: 0
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 1
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 1
+cpu cores	: 12
+apicid		: 2
+initial apicid	: 2
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 2
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 2
+cpu cores	: 12
+apicid		: 4
+initial apicid	: 4
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 3
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 3
+cpu cores	: 12
+apicid		: 6
+initial apicid	: 6
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 4
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 4
+cpu cores	: 12
+apicid		: 8
+initial apicid	: 8
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 5
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 5
+cpu cores	: 12
+apicid		: 10
+initial apicid	: 10
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 6
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 8
+cpu cores	: 12
+apicid		: 16
+initial apicid	: 16
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 7
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 9
+cpu cores	: 12
+apicid		: 18
+initial apicid	: 18
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 8
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 10
+cpu cores	: 12
+apicid		: 20
+initial apicid	: 20
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 9
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 11
+cpu cores	: 12
+apicid		: 22
+initial apicid	: 22
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 10
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 12
+cpu cores	: 12
+apicid		: 24
+initial apicid	: 24
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 11
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 13
+cpu cores	: 12
+apicid		: 26
+initial apicid	: 26
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 12
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 0
+cpu cores	: 12
+apicid		: 1
+initial apicid	: 1
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 13
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 1
+cpu cores	: 12
+apicid		: 3
+initial apicid	: 3
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 14
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 2
+cpu cores	: 12
+apicid		: 5
+initial apicid	: 5
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 15
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 3
+cpu cores	: 12
+apicid		: 7
+initial apicid	: 7
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 16
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 4
+cpu cores	: 12
+apicid		: 9
+initial apicid	: 9
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 17
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 5
+cpu cores	: 12
+apicid		: 11
+initial apicid	: 11
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 18
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 8
+cpu cores	: 12
+apicid		: 17
+initial apicid	: 17
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 19
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 9
+cpu cores	: 12
+apicid		: 19
+initial apicid	: 19
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 20
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 10
+cpu cores	: 12
+apicid		: 21
+initial apicid	: 21
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 21
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 11
+cpu cores	: 12
+apicid		: 23
+initial apicid	: 23
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 22
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 12
+cpu cores	: 12
+apicid		: 25
+initial apicid	: 25
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
+processor	: 23
+vendor_id	: GenuineIntel
+cpu family	: 6
+model		: 79
+model name	: Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz
+stepping	: 1
+microcode	: 0xb00002a
+cpu MHz		: 3199.951
+cache size	: 30720 KB
+physical id	: 0
+siblings	: 24
+core id		: 13
+cpu cores	: 12
+apicid		: 27
+initial apicid	: 27
+fpu		: yes
+fpu_exception	: yes
+cpuid level	: 20
+wp		: yes
+flags		: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnm [...]
+bugs		: cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass
+bogomips	: 5986.05
+clflush size	: 64
+cache_alignment	: 64
+address sizes	: 46 bits physical, 48 bits virtual
+power management:
+
diff --git a/pom.xml b/pom.xml
index 5c6d9eb..9903f5a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -65,6 +65,7 @@
     <module>bookkeeper-http</module>
     <module>stream/distributedlog</module>
     <module>tools</module>
+    <module>cpu-affinity</module>
     <module>metadata-drivers</module>
     <module>bookkeeper-dist</module>
     <module>shaded</module>
@@ -190,7 +191,7 @@
     <maven-shade-plugin.version>3.1.0</maven-shade-plugin.version>
     <maven-source-plugin.version>2.2.1</maven-source-plugin.version>
     <maven-surefire-plugin.version>2.21.0</maven-surefire-plugin.version>
-    <nar-maven-plugin.version>3.1.0</nar-maven-plugin.version>
+    <nar-maven-plugin.version>3.5.2</nar-maven-plugin.version>
     <os-maven-plugin.version>1.4.1.Final</os-maven-plugin.version>
     <protobuf-maven-plugin.version>0.5.0</protobuf-maven-plugin.version>
     <puppycrawl.checkstyle.version>6.19</puppycrawl.checkstyle.version>