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/12/12 09:55:39 UTC

[01/10] cassandra git commit: Rely on the JVM to handle OutOfMemoryErrors

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 3cd2c3c4e -> 02aba7343
  refs/heads/cassandra-3.0 d7329a639 -> dd187d105
  refs/heads/cassandra-3.11 88a41fb82 -> a7c45be93
  refs/heads/trunk 8e95534fb -> 17e602b7d


Rely on the JVM to handle OutOfMemoryErrors

patch by Benjamin Lerer; reviewed by Joshua McKenzie for CASSANDRA-13006


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

Branch: refs/heads/cassandra-2.2
Commit: 02aba7343ce300397ab672bbb1788aa8182d8a48
Parents: 3cd2c3c
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:21:05 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:21:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  9 ++-
 bin/cassandra                                   | 19 ++++-
 conf/cassandra-env.ps1                          | 10 +++
 conf/cassandra-env.sh                           | 12 +++
 .../apache/cassandra/service/StartupChecks.java | 74 ++++++++++++++++++
 .../org/apache/cassandra/utils/HeapUtils.java   | 82 ++++----------------
 .../cassandra/utils/JVMStabilityInspector.java  | 24 +++++-
 .../utils/JVMStabilityInspectorTest.java        | 31 +++++---
 9 files changed, 178 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1e81fd..5200eb1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.12
+ * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
  * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 
 2.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3bff458..5747941 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,8 +18,13 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.2 if you are upgrading
-      from a previous version.
+    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
+      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
+      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
+      options are used. See CASSANDRA-13006 for more details.
+    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
+      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
+      for more details.
 
 2.2.11
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/bin/cassandra
----------------------------------------------------------------------
diff --git a/bin/cassandra b/bin/cassandra
index 2dd0fe1..0e337e8 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -28,6 +28,7 @@
 #
 #   CLASSPATH -- A Java classpath containing everything necessary to run.
 #   JVM_OPTS -- Additional arguments to the JVM for heap size, etc
+#   JVM_ON_OUT_OF_MEMORY_ERROR_OPT -- The OnOutOfMemoryError JVM option if specified
 #   CASSANDRA_CONF -- Directory containing Cassandra configuration files.
 #
 # As a convenience, a fragment of shell is sourced in order to set one or
@@ -199,12 +200,22 @@ launch_service()
     # to close stdout/stderr, but it's up to us not to background.
     if [ "x$foreground" != "x" ]; then
         cassandra_parms="$cassandra_parms -Dcassandra-foreground=yes"
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        fi
     # Startup CassandraDaemon, background it, and write the pid.
     else
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
-        [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
-        true
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        fi
     fi
 
     return $?

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 321a9ca..7b4a632 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -390,6 +390,16 @@ Function SetCassandraEnvironment
     $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 
+    # stop the jvm on OutOfMemoryError as it can result in some data corruption
+    # uncomment the preferred option
+    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+
+    # print an heap histogram on OutOfMemoryError
+    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
     # Per-thread stack size.
     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index b519b76..7b1b8d3 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -204,6 +204,18 @@ fi
 
 startswith() { [ "${1#$2}" != "$1" ]; }
 
+# stop the jvm on OutOfMemoryError as it can result in some data corruption
+# uncomment the preferred option
+# For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+# on white spaces without taking quotes into account
+# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+# JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+# JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+
+# print an heap histogram on OutOfMemoryError
+# JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
 # Per-thread stack size.
 JVM_OPTS="$JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 34bc824..7ec16d1 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.service;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
 import java.nio.file.*;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
@@ -178,6 +180,78 @@ public class StartupChecks
             {
                 logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
             }
+            else
+            {
+                    checkOutOfMemoryHandling();
+            }
+        }
+
+        /**
+         * Checks that the JVM is configured to handle OutOfMemoryError
+         */
+        private void checkOutOfMemoryHandling()
+        {
+            int version = getJavaVersion();
+            int update = getUpdate();
+            // The ExitOnOutOfMemory and CrashOnOutOfMemory are supported since the version 7u101 and 8u92
+            boolean jreSupportExitOnOutOfMemory = version > 8
+                                                    || (version == 7 && update >= 101)
+                                                    || (version == 8 && update >= 92);
+            if (jreSupportExitOnOutOfMemory)
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError=", "-XX:+ExitOnOutOfMemoryError", "-XX:+CrashOnOutOfMemoryError"))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                                + " Use one of the following JVM options to configure the behavior on OutOfMemoryError: "
+                                + " -XX:+ExitOnOutOfMemoryError, -XX:+CrashOnOutOfMemoryError, or -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\"");
+            }
+            else
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError="))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                            + " Either upgrade your JRE to a version greater or equal to 8u92 and use -XX:+ExitOnOutOfMemoryError/-XX:+CrashOnOutOfMemoryError"
+                            + " or use -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\" on your current JRE.");
+            }
+        }
+
+        /**
+         * Returns the java version number for an Oracle JVM.
+         * @return the java version number
+         */
+        private int getJavaVersion()
+        {
+            String jreVersion = System.getProperty("java.version");
+            String version = jreVersion.startsWith("1.") ? jreVersion.substring(2, 3) // Pre 9 version
+                                                         : jreVersion.substring(0, jreVersion.indexOf('.'));
+            return Integer.parseInt(version);
+        }
+
+        /**
+         * Return the update number for an Oracle JVM.
+         * @return the update number
+         */
+        private int getUpdate()
+        {
+            String jreVersion = System.getProperty("java.version");
+            int updateSeparatorIndex = jreVersion.indexOf('_');
+            return Integer.parseInt(jreVersion.substring(updateSeparatorIndex + 1));
+        }
+
+        /**
+         * Checks if one of the specified options is being used.
+         * @param optionNames The name of the options to check
+         * @return {@code true} if one of the specified options is being used, {@code false} otherwise.
+         */
+        private boolean jvmOptionsContainsOneOf(String... optionNames)
+        {
+            RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
+            List<String> inputArguments = runtimeMxBean.getInputArguments();
+            for (String argument : inputArguments)
+            {
+                for (String optionName : optionNames)
+                    if (argument.startsWith(optionName))
+                        return true;
+            }
+            return false;
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 bfc8a0b..2d068de 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -19,11 +19,6 @@ package org.apache.cassandra.utils;
 
 import java.io.*;
 import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.text.StrBuilder;
@@ -32,7 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Utility to generate heap dumps.
+ * Utility to log heap histogram.
  *
  */
 public final class HeapUtils
@@ -43,54 +38,33 @@ public final class HeapUtils
      * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
      * or in the <code>CASSANDRA_HOME</code> directory.
      */
-    public static void generateHeapDump()
+    public static void logHeapHistogram()
     {
-        Long processId = getProcessId();
-        if (processId == null)
+        try
         {
-            logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-            return;
-        }
+            logger.info("Trying to log the heap histogram using jmap");
 
-        String heapDumpPath = getHeapDumpPathOption();
-        if (heapDumpPath == null)
-        {
-            String cassandraHome = System.getenv("CASSANDRA_HOME");
-            if (cassandraHome == null)
+            Long processId = getProcessId();
+            if (processId == null)
             {
+                logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                 return;
             }
 
-            heapDumpPath = cassandraHome;
-        }
+            String jmapPath = getJmapPath();
 
-        Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-        if (Files.isDirectory(dumpPath))
-        {
-            dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-        }
+            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
+            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
 
-        String jmapPath = getJmapPath();
+            String[] histoCommands = new String[] {jmapCommand,
+                    "-histo",
+                    processId.toString()};
 
-        // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-        String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
-
-        String[] dumpCommands = new String[] {jmapCommand,
-                                              "-dump:format=b,file=" + dumpPath,
-                                              processId.toString()};
-
-        // Lets also log the Heap histogram
-        String[] histoCommands = new String[] {jmapCommand,
-                                               "-histo",
-                                               processId.toString()};
-        try
-        {
-            logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
             logProcessOutput(Runtime.getRuntime().exec(histoCommands));
         }
-        catch (IOException e)
+        catch (Throwable e)
         {
-            logger.error("The heap dump could not be generated due to the following error: ", e);
+            logger.error("The heap histogram could not be generated due to the following error: ", e);
         }
     }
 
@@ -137,32 +111,6 @@ public final class HeapUtils
     }
 
     /**
-     * Retrieves the value of the <code>HeapDumpPath</code> JVM option.
-     * @return the value of the <code>HeapDumpPath</code> JVM option or <code>null</code> if the value has not been
-     * specified.
-     */
-    private static String getHeapDumpPathOption()
-    {
-        RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
-        List<String> inputArguments = runtimeMxBean.getInputArguments();
-        String heapDumpPathOption = null;
-        for (String argument : inputArguments)
-        {
-            if (argument.startsWith("-XX:HeapDumpPath="))
-            {
-                heapDumpPathOption = argument;
-                // We do not break in case the option has been specified several times.
-                // In general it seems that JVMs use the right-most argument as the winner.
-            }
-        }
-
-        if (heapDumpPathOption == null)
-            return null;
-
-        return heapDumpPathOption.substring(17, heapDumpPathOption.length());
-    }
-
-    /**
      * Retrieves the process ID or <code>null</code> if the process ID cannot be retrieved.
      * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index f8cb775..0196b04 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.net.SocketException;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,6 +39,8 @@ public final class JVMStabilityInspector
     private static final Logger logger = LoggerFactory.getLogger(JVMStabilityInspector.class);
     private static Killer killer = new Killer();
 
+    private static Object lock = new Object();
+    private static boolean printingHeapHistogram;
 
     private JVMStabilityInspector() {}
 
@@ -52,8 +55,25 @@ public final class JVMStabilityInspector
         boolean isUnstable = false;
         if (t instanceof OutOfMemoryError)
         {
-            isUnstable = true;
-            HeapUtils.generateHeapDump();
+            if (Boolean.getBoolean("cassandra.printHeapHistogramOnOutOfMemoryError"))
+            {
+                // We want to avoid printing multiple time the heap histogram if multiple OOM errors happen in a short
+                // time span.
+                synchronized(lock)
+                {
+                    if (printingHeapHistogram)
+                        return;
+                    printingHeapHistogram = true;
+                }
+                HeapUtils.logHeapHistogram();
+            }
+
+            logger.error("OutOfMemory error letting the JVM handle the error:", t);
+
+            StorageService.instance.removeShutdownHook();
+            // We let the JVM handle the error. The startup checks should have warned the user if it did not configure
+            // the JVM behavior in case of OOM (CASSANDRA-13006).
+            throw (OutOfMemoryError) t;
         }
 
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index 7142f97..f96ac6e 100644
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@ -20,14 +20,19 @@ package org.apache.cassandra.utils;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+
+import static java.util.Arrays.asList;
+
 import org.junit.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.SocketException;
+import java.util.Arrays;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class JVMStabilityInspectorTest
 {
@@ -45,10 +50,6 @@ public class JVMStabilityInspectorTest
             JVMStabilityInspector.inspectThrowable(new IOException());
             assertFalse(killerForTests.wasKilled());
 
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new OutOfMemoryError());
-            assertTrue(killerForTests.wasKilled());
-
             DatabaseDescriptor.setDiskFailurePolicy(Config.DiskFailurePolicy.die);
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new FSReadError(new IOException(), "blah"));
@@ -62,11 +63,6 @@ public class JVMStabilityInspectorTest
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new Exception(new IOException()));
             assertFalse(killerForTests.wasKilled());
-
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new Exception(new OutOfMemoryError()));
-            assertTrue(killerForTests.wasKilled());
-
         }
         finally
         {
@@ -77,6 +73,23 @@ public class JVMStabilityInspectorTest
     }
 
     @Test
+    public void testOutOfMemoryHandling()
+    {
+        for (Throwable oom : asList(new OutOfMemoryError(), new Exception(new OutOfMemoryError())))
+        {
+            try
+            {
+                JVMStabilityInspector.inspectThrowable(oom);
+                fail("The JVMStabilityInspector should delegate the handling of OutOfMemoryErrors to the JVM");
+            }
+            catch (OutOfMemoryError e)
+            {
+                assertTrue(true);
+            }
+        }
+    }
+
+    @Test
     public void fileHandleTest()
     {
         KillerForTests killerForTests = new KillerForTests();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[10/10] cassandra git commit: Merge branch cassandra-3.11 into trunk

Posted by bl...@apache.org.
Merge branch cassandra-3.11 into trunk


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

Branch: refs/heads/trunk
Commit: 17e602b7d248c61634c31c93b7d244f35cfcefd3
Parents: 8e95534 a7c45be
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:52:23 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:52:23 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 NEWS.txt                                        |  13 +
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 914 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 638 insertions(+), 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e602b7/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e602b7/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 7bb9152,0c32278..259507e
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,103 -13,27 +13,116 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 +4.0
 +===
 +
 +New features
 +------------
 +   - An experimental option to compare all merkle trees together has been added - for example, in
 +     a 3 node cluster with 2 replicas identical and 1 out-of-date, with this option enabled, the
 +     out-of-date replica will only stream a single copy from up-to-date replica. Enable it by adding
 +     "-os" to nodetool repair. See CASSANDRA-3200.
 +   - The currentTimestamp, currentDate, currentTime and currentTimeUUID functions have been added.
 +     See CASSANDRA-13132
 +   - Support for arithmetic operations between `timestamp`/`date` and `duration` has been added.
 +     See CASSANDRA-11936
 +   - Support for arithmetic operations on number has been added. See CASSANDRA-11935
 +   - Preview expected streaming required for a repair (nodetool repair --preview), and validate the
 +     consistency of repaired data between nodes (nodetool repair --validate). See CASSANDRA-13257
 +   - Support for selecting Map values and Set elements has been added for SELECT queries. See CASSANDRA-7396
 +   - Change-Data-Capture has been modified to make CommitLogSegments available
 +     immediately upon creation via hard-linking the files. This means that incomplete
 +     segments will be available in cdc_raw rather than fully flushed. See documentation
 +     and CASSANDRA-12148 for more detail.
 +   - The initial build of materialized views can be parallelized. The number of concurrent builder
 +     threads is specified by the property `cassandra.yaml:concurrent_materialized_view_builders`.
 +     This property can be modified at runtime through both JMX and the new `setconcurrentviewbuilders`
 +     and `getconcurrentviewbuilders` nodetool commands. See CASSANDRA-12245 for more details.
 +   - There is now a binary full query log based on Chronicle Queue that can be controlled using
 +     nodetool enablefullquerylog, disablefullquerylog, and resetfullquerylog. The log 
 +     contains all queries invoked, approximate time they were invoked, any parameters necessary
 +     to bind wildcard values, and all query options. A human readable version of the log can be
 +     dumped or tailed using the new bin/fqltool utility. The full query log is designed to be safe
 +     to use in production and limits utilization of heap memory and disk space with limits
 +     you can specify when enabling the log.
 +     See nodetool and fqltool help text for more information.
 +   - SSTableDump now supports the -l option to output each partition as it's own json object
 +     See CASSANDRA-13848 for more detail
 +
 +Upgrading
 +---------
 +    - Cassandra 4.0 removed support for COMPACT STORAGE tables. All Compact Tables
 +      have to be migrated using `ALTER ... DROP COMPACT STORAGE` statement in 3.0/3.11.
 +      Cassandra starting 4.0 will not start if flags indicate that the table is non-CQL.
 +      Syntax for creating compact tables is also deprecated.
 +    - Support for legacy auth tables in the system_auth keyspace (users,
 +      permissions, credentials) and the migration code has been removed. Migration
 +      of these legacy auth tables must have been completed before the upgrade to
 +      4.0 and the legacy tables must have been removed. See the 'Upgrading' section
 +      for version 2.2 for migration instructions.
 +    - Cassandra 4.0 removed support for the deprecated Thrift interface. Amongst
 +      other things, this implies the removal of all yaml options related to thrift
 +      ('start_rpc', rpc_port, ...).
 +    - Cassandra 4.0 removed support for any pre-3.0 format. This means you
 +      cannot upgrade from a 2.x version to 4.0 directly, you have to upgrade to
 +      a 3.0.x/3.x version first (and run upgradesstable). In particular, this
 +      mean Cassandra 4.0 cannot load or read pre-3.0 sstables in any way: you
 +      will need to upgrade those sstable in 3.0.x/3.x first.
 +    - Upgrades from 3.0.x or 3.x are supported since 3.0.13 or 3.11.0, previous
 +      versions will causes issues during rolling upgrades (CASSANDRA-13274).
 +    - Cassandra will no longer allow invalid keyspace replication options, such
 +      as invalid datacenter names for NetworkTopologyStrategy. Operators MUST
 +      add new nodes to a datacenter before they can set set ALTER or CREATE
 +      keyspace replication policies using that datacenter. Existing keyspaces
 +      will continue to operate, but CREATE and ALTER will validate that all
 +      datacenters specified exist in the cluster.
 +    - Cassandra 4.0 fixes a problem with incremental repair which caused repaired
 +      data to be inconsistent between nodes. The fix changes the behavior of both
 +      full and incremental repairs. For full repairs, data is no longer marked
 +      repaired. For incremental repairs, anticompaction is run at the beginning
 +      of the repair, instead of at the end. If incremental repair was being used
 +      prior to upgrading, a full repair should be run after upgrading to resolve
 +      any inconsistencies.
 +    - Config option index_interval has been removed (it was deprecated since 2.0)
 +    - Deprecated repair JMX APIs are removed.
 +    - The version of snappy-java has been upgraded to 1.1.2.6
 +	- the miniumum value for internode message timeouts is 10ms. Previously, any
 +	  positive value was allowed. See cassandra.yaml entries like
 +	  read_request_timeout_in_ms for more details.
 +	- Cassandra 4.0 allows a single port to be used for both secure and insecure
 +	  connections between cassandra nodes (CASSANDRA-10404). See the yaml for
 +	  specific property changes, and see the security doc for full details.
 +    - Due to the parallelization of the initial build of materialized views,
 +      the per token range view building status is stored in the new table
 +      `system.view_builds_in_progress`. The old table `system.views_builds_in_progress`
 +      is no longer used and can be removed. See CASSANDRA-12245 for more details.
 +	- Config option commitlog_sync_batch_window_in_ms has been deprecated as it's
 +	  documentation has been incorrect and the setting itself near useless.
 +	  Batch mode remains a valid commit log mode, however.
 +	- There is a new commit log mode, group, which is similar to batch mode
 +	  but blocks for up to a configurable number of milliseconds between disk flushes.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
+ 3.11.2
+ ======
+ 
+ Upgrading
+ ---------
+     - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
+       rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
+       are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
+       options are used. See CASSANDRA-13006 for more details
+     - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
+       set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
+       for more details.
+ 
 -Materialized Views
 --------------------
 -   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 -     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 -     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 -   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 -     views
 -
  3.11.1
  ======
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e602b7/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index 806eabc,49d03ce..72d632b
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -1,381 -1,381 +1,381 @@@
- #
- # 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.
- 
- # NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
- 
- #-----------------------------------------------------------------------------
- Function SetCassandraHome()
- {
-     if (! $env:CASSANDRA_HOME)
-     {
-         $cwd = [System.IO.Directory]::GetCurrentDirectory()
-         $cwd = Split-Path $cwd -parent
-         $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
-     }
- }
- 
- #-----------------------------------------------------------------------------
- Function SetCassandraMain()
- {
-     if (! $env:CASSANDRA_MAIN)
-     {
-         $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
-     }
- }
- 
- #-----------------------------------------------------------------------------
- Function BuildClassPath
- {
-     $cp = """$env:CASSANDRA_HOME\conf"""
-     foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
-     {
-         $file = $file -replace "\\", "/"
-         $cp = $cp + ";" + """$file"""
-     }
- 
-     # Add build/classes/main so it works in development
-     $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"""
-     $env:CLASSPATH=$cp
- }
- 
- #-----------------------------------------------------------------------------
- Function CalculateHeapSizes
- {
-     # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
- 
-     $osInfo = Get-WmiObject -class "Win32_computersystem"
-     $autoPage = $osInfo.AutomaticManagedPageFile
- 
-     if ($autoPage)
-     {
-         echo "*---------------------------------------------------------------------*"
-         echo "*---------------------------------------------------------------------*"
-         echo ""
-         echo "    WARNING!  Automatic page file configuration detected."
-         echo "    It is recommended that you disable swap when running Cassandra"
-         echo "    for performance and stability reasons."
-         echo ""
-         echo "*---------------------------------------------------------------------*"
-         echo "*---------------------------------------------------------------------*"
-     }
-     else
-     {
-         $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
-         $pageFileCount = $PageFileInfo.Count
-         if ($pageFileInfo)
-         {
-             $files = @()
-             $sizes = @()
-             $hasSizes = $FALSE
- 
-             # PageFileCount isn't populated and obj comes back as single if there's only 1
-             if ([string]::IsNullOrEmpty($PageFileCount))
-             {
-                 $PageFileCount = 1
-                 $files += $PageFileInfo.Name
-                 if ($PageFileInfo.MaximumSize -ne 0)
-                 {
-                     $hasSizes = $TRUE
-                     $sizes += $PageFileInfo.MaximumSize
-                 }
-             }
-             else
-             {
-                 for ($i = 0; $i -le $PageFileCount; $i++)
-                 {
-                     $files += $PageFileInfo[$i].Name
-                     if ($PageFileInfo[$i].MaximumSize -ne 0)
-                     {
-                         $hasSizes = $TRUE
-                         $sizes += $PageFileInfo[$i].MaximumSize
-                     }
-                 }
-             }
- 
-             echo "*---------------------------------------------------------------------*"
-             echo "*---------------------------------------------------------------------*"
-             echo ""
-             echo "    WARNING!  $PageFileCount swap file(s) detected"
-             for ($i = 0; $i -lt $PageFileCount; $i++)
-             {
-                 $toPrint = "        Name: " + $files[$i]
-                 if ($hasSizes)
-                 {
-                     $toPrint = $toPrint + " Size: " + $sizes[$i]
-                     $toPrint = $toPrint -replace [Environment]::NewLine, ""
-                 }
-                 echo $toPrint
-             }
-             echo "    It is recommended that you disable swap when running Cassandra"
-             echo "    for performance and stability reasons."
-             echo ""
-             echo "*---------------------------------------------------------------------*"
-             echo "*---------------------------------------------------------------------*"
-         }
-     }
- 
-     # Validate that we need to run this function and that our config is good
-     if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
-     {
-         return
-     }
- 
-     if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
-     {
-         echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
-         exit 1
-     }
- 
-     $memObject = Get-WMIObject -class win32_physicalmemory
-     if ($memObject -eq $null)
-     {
-         echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
-         $env:MAX_HEAP_SIZE = "2048M"
-         $env:HEAP_NEWSIZE = "512M"
-         return
-     }
- 
-     $memory = ($memObject | Measure-Object Capacity -Sum).sum
-     $memoryMB = [Math]::Truncate($memory / (1024*1024))
- 
-     $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
-     $systemCores = $cpu.NumberOfLogicalProcessors
- 
-     # set max heap size based on the following
-     # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
-     # calculate 1/2 ram and cap to 1024MB
-     # calculate 1/4 ram and cap to 8192MB
-     # pick the max
-     $halfMem = [Math]::Truncate($memoryMB / 2)
-     $quarterMem = [Math]::Truncate($halfMem / 2)
- 
-     if ($halfMem -gt 1024)
-     {
-         $halfMem = 1024
-     }
-     if ($quarterMem -gt 8192)
-     {
-         $quarterMem = 8192
-     }
- 
-     $maxHeapMB = ""
-     if ($halfMem -gt $quarterMem)
-     {
-         $maxHeapMB = $halfMem
-     }
-     else
-     {
-         $maxHeapMB = $quarterMem
-     }
-     $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
- 
-     # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
-     $maxYGPerCore = 100
-     $maxYGTotal = $maxYGPerCore * $systemCores
-     $desiredYG = [Math]::Truncate($maxHeapMB / 4)
- 
-     if ($desiredYG -gt $maxYGTotal)
-     {
-         $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
-     }
-     else
-     {
-         $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
-     }
- }
- 
- #-----------------------------------------------------------------------------
- Function ParseJVMInfo
- {
-     # grab info about the JVM
-     $pinfo = New-Object System.Diagnostics.ProcessStartInfo
-     $pinfo.FileName = "$env:JAVA_BIN"
-     $pinfo.RedirectStandardError = $true
-     $pinfo.RedirectStandardOutput = $true
-     $pinfo.UseShellExecute = $false
-     $pinfo.Arguments = "-d64 -version"
-     $p = New-Object System.Diagnostics.Process
-     $p.StartInfo = $pinfo
-     $p.Start() | Out-Null
-     $p.WaitForExit()
-     $stderr = $p.StandardError.ReadToEnd()
- 
-     $env:JVM_ARCH = "64-bit"
- 
-     if ($stderr.Contains("Error"))
-     {
-         # 32-bit JVM. re-run w/out -d64
-         echo "Failed 64-bit check. Re-running to get version from 32-bit"
-         $pinfo.Arguments = "-version"
-         $p = New-Object System.Diagnostics.Process
-         $p.StartInfo = $pinfo
-         $p.Start() | Out-Null
-         $p.WaitForExit()
-         $stderr = $p.StandardError.ReadToEnd()
-         $env:JVM_ARCH = "32-bit"
-     }
- 
-     $sa = $stderr.Split("""")
-     $env:JVM_VERSION = $sa[1]
- 
-     if ($stderr.Contains("OpenJDK"))
-     {
-         $env:JVM_VENDOR = "OpenJDK"
-     }
-     elseif ($stderr.Contains("Java(TM)"))
-     {
-         $env:JVM_VENDOR = "Oracle"
-     }
-     else
-     {
-         $JVM_VENDOR = "other"
-     }
- 
-     $pa = $sa[1].Split("_")
-     $subVersion = $pa[1]
-     # Deal with -b (build) versions
-     if ($subVersion -contains '-')
-     {
-         $patchAndBuild = $subVersion.Split("-")
-         $subVersion = $patchAndBuild[0]
-     }
-     $env:JVM_PATCH_VERSION = $subVersion
- }
- 
- #-----------------------------------------------------------------------------
- Function SetCassandraEnvironment
- {
-     if (Test-Path Env:\JAVA_HOME)
-     {
-         $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
-     }
-     elseif (Get-Command "java.exe")
-     {
-         $env:JAVA_BIN = "java.exe"
-     }
-     else
-     {
-         echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
-         exit
-     }
-     SetCassandraHome
-     $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
-     $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
- 
-     $logdir = "$env:CASSANDRA_HOME\logs"
-     $storagedir = "$env:CASSANDRA_HOME\data"
-     $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
- 
-     SetCassandraMain
-     BuildClassPath
- 
-     # Override these to set the amount of memory to allocate to the JVM at
-     # start-up. For production use you may wish to adjust this for your
-     # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
-     # to the Java heap. HEAP_NEWSIZE refers to the size of the young
-     # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
-     # or not (if you set one, set the other).
-     #
-     # The main trade-off for the young generation is that the larger it
-     # is, the longer GC pause times will be. The shorter it is, the more
-     # expensive GC will be (usually).
-     #
-     # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
-     # times. If in doubt, and if you do not particularly want to tweak, go
-     # 100 MB per physical CPU core.
- 
-     #GC log path has to be defined here since it needs to find CASSANDRA_HOME
-     $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
- 
-     # Read user-defined JVM options from jvm.options file
-     $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
-     for ($i = 0; $i -lt $content.Count; $i++)
-     {
-         $line = $content[$i]
-         if ($line.StartsWith("-"))
-         {
-             $env:JVM_OPTS = "$env:JVM_OPTS $line"
-         }
-     }
- 
-     $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
-     $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
-     $defined_xms = $env:JVM_OPTS -like '*Xms*'
-     $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
- 
-     #$env:MAX_HEAP_SIZE="4096M"
-     #$env:HEAP_NEWSIZE="800M"
-     CalculateHeapSizes
- 
-     ParseJVMInfo
- 
-     # We only set -Xms and -Xmx if they were not defined on jvm.options file
-     # If defined, both Xmx and Xms should be defined together.
-     if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
-     }
-     elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
-         exit
-     }
- 
-     # We only set -Xmn flag if it was not defined in jvm.options file
-     # and if the CMS GC is being used
-     # If defined, both Xmn and Xmx should be defined together.
-     if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
-         exit
-     }
-     elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
-     }
- 
-     if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
-     }
- 
-     # Add sigar env - see Cassandra-7838
-     $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
- 
-     # Confirm we're on high performance power plan, warn if not
-     # Change to $true to suppress this warning
-     $suppressPowerWarning = $false
-     if (!$suppressPowerWarning)
-     {
-         $currentProfile = powercfg /GETACTIVESCHEME
-         if (!$currentProfile.Contains("High performance"))
-         {
-             echo "*---------------------------------------------------------------------*"
-             echo "*---------------------------------------------------------------------*"
-             echo ""
-             echo "    WARNING! Detected a power profile other than High Performance."
-             echo "    Performance of this node will suffer."
-             echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
-             echo ""
-             echo "*---------------------------------------------------------------------*"
-             echo "*---------------------------------------------------------------------*"
-         }
-     }
- 
+ #
+ # 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.
+ 
+ # NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
+ 
+ #-----------------------------------------------------------------------------
+ Function SetCassandraHome()
+ {
+     if (! $env:CASSANDRA_HOME)
+     {
+         $cwd = [System.IO.Directory]::GetCurrentDirectory()
+         $cwd = Split-Path $cwd -parent
+         $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
+     }
+ }
+ 
+ #-----------------------------------------------------------------------------
+ Function SetCassandraMain()
+ {
+     if (! $env:CASSANDRA_MAIN)
+     {
+         $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
+     }
+ }
+ 
+ #-----------------------------------------------------------------------------
+ Function BuildClassPath
+ {
+     $cp = """$env:CASSANDRA_HOME\conf"""
+     foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
+     {
+         $file = $file -replace "\\", "/"
+         $cp = $cp + ";" + """$file"""
+     }
+ 
+     # Add build/classes/main so it works in development
 -    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
++    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"""
+     $env:CLASSPATH=$cp
+ }
+ 
+ #-----------------------------------------------------------------------------
+ Function CalculateHeapSizes
+ {
+     # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
+ 
+     $osInfo = Get-WmiObject -class "Win32_computersystem"
+     $autoPage = $osInfo.AutomaticManagedPageFile
+ 
+     if ($autoPage)
+     {
+         echo "*---------------------------------------------------------------------*"
+         echo "*---------------------------------------------------------------------*"
+         echo ""
+         echo "    WARNING!  Automatic page file configuration detected."
+         echo "    It is recommended that you disable swap when running Cassandra"
+         echo "    for performance and stability reasons."
+         echo ""
+         echo "*---------------------------------------------------------------------*"
+         echo "*---------------------------------------------------------------------*"
+     }
+     else
+     {
+         $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
+         $pageFileCount = $PageFileInfo.Count
+         if ($pageFileInfo)
+         {
+             $files = @()
+             $sizes = @()
+             $hasSizes = $FALSE
+ 
+             # PageFileCount isn't populated and obj comes back as single if there's only 1
+             if ([string]::IsNullOrEmpty($PageFileCount))
+             {
+                 $PageFileCount = 1
+                 $files += $PageFileInfo.Name
+                 if ($PageFileInfo.MaximumSize -ne 0)
+                 {
+                     $hasSizes = $TRUE
+                     $sizes += $PageFileInfo.MaximumSize
+                 }
+             }
+             else
+             {
+                 for ($i = 0; $i -le $PageFileCount; $i++)
+                 {
+                     $files += $PageFileInfo[$i].Name
+                     if ($PageFileInfo[$i].MaximumSize -ne 0)
+                     {
+                         $hasSizes = $TRUE
+                         $sizes += $PageFileInfo[$i].MaximumSize
+                     }
+                 }
+             }
+ 
+             echo "*---------------------------------------------------------------------*"
+             echo "*---------------------------------------------------------------------*"
+             echo ""
+             echo "    WARNING!  $PageFileCount swap file(s) detected"
+             for ($i = 0; $i -lt $PageFileCount; $i++)
+             {
+                 $toPrint = "        Name: " + $files[$i]
+                 if ($hasSizes)
+                 {
+                     $toPrint = $toPrint + " Size: " + $sizes[$i]
+                     $toPrint = $toPrint -replace [Environment]::NewLine, ""
+                 }
+                 echo $toPrint
+             }
+             echo "    It is recommended that you disable swap when running Cassandra"
+             echo "    for performance and stability reasons."
+             echo ""
+             echo "*---------------------------------------------------------------------*"
+             echo "*---------------------------------------------------------------------*"
+         }
+     }
+ 
+     # Validate that we need to run this function and that our config is good
+     if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
+     {
+         return
+     }
+ 
+     if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
+     {
+         echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
+         exit 1
+     }
+ 
+     $memObject = Get-WMIObject -class win32_physicalmemory
+     if ($memObject -eq $null)
+     {
+         echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
+         $env:MAX_HEAP_SIZE = "2048M"
+         $env:HEAP_NEWSIZE = "512M"
+         return
+     }
+ 
+     $memory = ($memObject | Measure-Object Capacity -Sum).sum
+     $memoryMB = [Math]::Truncate($memory / (1024*1024))
+ 
+     $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
+     $systemCores = $cpu.NumberOfLogicalProcessors
+ 
+     # set max heap size based on the following
+     # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
+     # calculate 1/2 ram and cap to 1024MB
+     # calculate 1/4 ram and cap to 8192MB
+     # pick the max
+     $halfMem = [Math]::Truncate($memoryMB / 2)
+     $quarterMem = [Math]::Truncate($halfMem / 2)
+ 
+     if ($halfMem -gt 1024)
+     {
+         $halfMem = 1024
+     }
+     if ($quarterMem -gt 8192)
+     {
+         $quarterMem = 8192
+     }
+ 
+     $maxHeapMB = ""
+     if ($halfMem -gt $quarterMem)
+     {
+         $maxHeapMB = $halfMem
+     }
+     else
+     {
+         $maxHeapMB = $quarterMem
+     }
+     $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
+ 
+     # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
+     $maxYGPerCore = 100
+     $maxYGTotal = $maxYGPerCore * $systemCores
+     $desiredYG = [Math]::Truncate($maxHeapMB / 4)
+ 
+     if ($desiredYG -gt $maxYGTotal)
+     {
+         $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
+     }
+     else
+     {
+         $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
+     }
+ }
+ 
+ #-----------------------------------------------------------------------------
+ Function ParseJVMInfo
+ {
+     # grab info about the JVM
+     $pinfo = New-Object System.Diagnostics.ProcessStartInfo
+     $pinfo.FileName = "$env:JAVA_BIN"
+     $pinfo.RedirectStandardError = $true
+     $pinfo.RedirectStandardOutput = $true
+     $pinfo.UseShellExecute = $false
+     $pinfo.Arguments = "-d64 -version"
+     $p = New-Object System.Diagnostics.Process
+     $p.StartInfo = $pinfo
+     $p.Start() | Out-Null
+     $p.WaitForExit()
+     $stderr = $p.StandardError.ReadToEnd()
+ 
+     $env:JVM_ARCH = "64-bit"
+ 
+     if ($stderr.Contains("Error"))
+     {
+         # 32-bit JVM. re-run w/out -d64
+         echo "Failed 64-bit check. Re-running to get version from 32-bit"
+         $pinfo.Arguments = "-version"
+         $p = New-Object System.Diagnostics.Process
+         $p.StartInfo = $pinfo
+         $p.Start() | Out-Null
+         $p.WaitForExit()
+         $stderr = $p.StandardError.ReadToEnd()
+         $env:JVM_ARCH = "32-bit"
+     }
+ 
+     $sa = $stderr.Split("""")
+     $env:JVM_VERSION = $sa[1]
+ 
+     if ($stderr.Contains("OpenJDK"))
+     {
+         $env:JVM_VENDOR = "OpenJDK"
+     }
+     elseif ($stderr.Contains("Java(TM)"))
+     {
+         $env:JVM_VENDOR = "Oracle"
+     }
+     else
+     {
+         $JVM_VENDOR = "other"
+     }
+ 
+     $pa = $sa[1].Split("_")
+     $subVersion = $pa[1]
+     # Deal with -b (build) versions
+     if ($subVersion -contains '-')
+     {
+         $patchAndBuild = $subVersion.Split("-")
+         $subVersion = $patchAndBuild[0]
+     }
+     $env:JVM_PATCH_VERSION = $subVersion
+ }
+ 
+ #-----------------------------------------------------------------------------
+ Function SetCassandraEnvironment
+ {
+     if (Test-Path Env:\JAVA_HOME)
+     {
+         $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
+     }
+     elseif (Get-Command "java.exe")
+     {
+         $env:JAVA_BIN = "java.exe"
+     }
+     else
+     {
+         echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
+         exit
+     }
+     SetCassandraHome
+     $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
+     $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
+ 
+     $logdir = "$env:CASSANDRA_HOME\logs"
+     $storagedir = "$env:CASSANDRA_HOME\data"
+     $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
+ 
+     SetCassandraMain
+     BuildClassPath
+ 
+     # Override these to set the amount of memory to allocate to the JVM at
+     # start-up. For production use you may wish to adjust this for your
+     # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
+     # to the Java heap. HEAP_NEWSIZE refers to the size of the young
+     # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
+     # or not (if you set one, set the other).
+     #
+     # The main trade-off for the young generation is that the larger it
+     # is, the longer GC pause times will be. The shorter it is, the more
+     # expensive GC will be (usually).
+     #
+     # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
+     # times. If in doubt, and if you do not particularly want to tweak, go
+     # 100 MB per physical CPU core.
+ 
+     #GC log path has to be defined here since it needs to find CASSANDRA_HOME
+     $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
+ 
+     # Read user-defined JVM options from jvm.options file
+     $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
+     for ($i = 0; $i -lt $content.Count; $i++)
+     {
+         $line = $content[$i]
+         if ($line.StartsWith("-"))
+         {
+             $env:JVM_OPTS = "$env:JVM_OPTS $line"
+         }
+     }
+ 
+     $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
+     $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
+     $defined_xms = $env:JVM_OPTS -like '*Xms*'
+     $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
+ 
+     #$env:MAX_HEAP_SIZE="4096M"
+     #$env:HEAP_NEWSIZE="800M"
+     CalculateHeapSizes
+ 
+     ParseJVMInfo
+ 
+     # We only set -Xms and -Xmx if they were not defined on jvm.options file
+     # If defined, both Xmx and Xms should be defined together.
+     if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
+     {
+         $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
+         $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
+     }
+     elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
+     {
+         echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
+         exit
+     }
+ 
+     # We only set -Xmn flag if it was not defined in jvm.options file
+     # and if the CMS GC is being used
+     # If defined, both Xmn and Xmx should be defined together.
+     if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
+     {
+         echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
+         exit
+     }
+     elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
+     {
+         $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
+     }
+ 
+     if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
+     {
+         $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
+     }
+ 
+     # Add sigar env - see Cassandra-7838
+     $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
+ 
+     # Confirm we're on high performance power plan, warn if not
+     # Change to $true to suppress this warning
+     $suppressPowerWarning = $false
+     if (!$suppressPowerWarning)
+     {
+         $currentProfile = powercfg /GETACTIVESCHEME
+         if (!$currentProfile.Contains("High performance"))
+         {
+             echo "*---------------------------------------------------------------------*"
+             echo "*---------------------------------------------------------------------*"
+             echo ""
+             echo "    WARNING! Detected a power profile other than High Performance."
+             echo "    Performance of this node will suffer."
+             echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
+             echo ""
+             echo "*---------------------------------------------------------------------*"
+             echo "*---------------------------------------------------------------------*"
+         }
+     }
+ 
      # provides hints to the JIT compiler
      $env:JVM_OPTS = "$env:JVM_OPTS -XX:CompileCommandFile=$env:CASSANDRA_CONF\hotspot_compiler"
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e602b7/conf/cassandra-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e602b7/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[02/10] cassandra git commit: Rely on the JVM to handle OutOfMemoryErrors

Posted by bl...@apache.org.
Rely on the JVM to handle OutOfMemoryErrors

patch by Benjamin Lerer; reviewed by Joshua McKenzie for CASSANDRA-13006


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

Branch: refs/heads/cassandra-3.0
Commit: 02aba7343ce300397ab672bbb1788aa8182d8a48
Parents: 3cd2c3c
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:21:05 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:21:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  9 ++-
 bin/cassandra                                   | 19 ++++-
 conf/cassandra-env.ps1                          | 10 +++
 conf/cassandra-env.sh                           | 12 +++
 .../apache/cassandra/service/StartupChecks.java | 74 ++++++++++++++++++
 .../org/apache/cassandra/utils/HeapUtils.java   | 82 ++++----------------
 .../cassandra/utils/JVMStabilityInspector.java  | 24 +++++-
 .../utils/JVMStabilityInspectorTest.java        | 31 +++++---
 9 files changed, 178 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1e81fd..5200eb1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.12
+ * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
  * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 
 2.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3bff458..5747941 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,8 +18,13 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.2 if you are upgrading
-      from a previous version.
+    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
+      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
+      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
+      options are used. See CASSANDRA-13006 for more details.
+    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
+      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
+      for more details.
 
 2.2.11
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/bin/cassandra
----------------------------------------------------------------------
diff --git a/bin/cassandra b/bin/cassandra
index 2dd0fe1..0e337e8 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -28,6 +28,7 @@
 #
 #   CLASSPATH -- A Java classpath containing everything necessary to run.
 #   JVM_OPTS -- Additional arguments to the JVM for heap size, etc
+#   JVM_ON_OUT_OF_MEMORY_ERROR_OPT -- The OnOutOfMemoryError JVM option if specified
 #   CASSANDRA_CONF -- Directory containing Cassandra configuration files.
 #
 # As a convenience, a fragment of shell is sourced in order to set one or
@@ -199,12 +200,22 @@ launch_service()
     # to close stdout/stderr, but it's up to us not to background.
     if [ "x$foreground" != "x" ]; then
         cassandra_parms="$cassandra_parms -Dcassandra-foreground=yes"
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        fi
     # Startup CassandraDaemon, background it, and write the pid.
     else
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
-        [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
-        true
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        fi
     fi
 
     return $?

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 321a9ca..7b4a632 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -390,6 +390,16 @@ Function SetCassandraEnvironment
     $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 
+    # stop the jvm on OutOfMemoryError as it can result in some data corruption
+    # uncomment the preferred option
+    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+
+    # print an heap histogram on OutOfMemoryError
+    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
     # Per-thread stack size.
     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index b519b76..7b1b8d3 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -204,6 +204,18 @@ fi
 
 startswith() { [ "${1#$2}" != "$1" ]; }
 
+# stop the jvm on OutOfMemoryError as it can result in some data corruption
+# uncomment the preferred option
+# For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+# on white spaces without taking quotes into account
+# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+# JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+# JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+
+# print an heap histogram on OutOfMemoryError
+# JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
 # Per-thread stack size.
 JVM_OPTS="$JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 34bc824..7ec16d1 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.service;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
 import java.nio.file.*;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
@@ -178,6 +180,78 @@ public class StartupChecks
             {
                 logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
             }
+            else
+            {
+                    checkOutOfMemoryHandling();
+            }
+        }
+
+        /**
+         * Checks that the JVM is configured to handle OutOfMemoryError
+         */
+        private void checkOutOfMemoryHandling()
+        {
+            int version = getJavaVersion();
+            int update = getUpdate();
+            // The ExitOnOutOfMemory and CrashOnOutOfMemory are supported since the version 7u101 and 8u92
+            boolean jreSupportExitOnOutOfMemory = version > 8
+                                                    || (version == 7 && update >= 101)
+                                                    || (version == 8 && update >= 92);
+            if (jreSupportExitOnOutOfMemory)
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError=", "-XX:+ExitOnOutOfMemoryError", "-XX:+CrashOnOutOfMemoryError"))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                                + " Use one of the following JVM options to configure the behavior on OutOfMemoryError: "
+                                + " -XX:+ExitOnOutOfMemoryError, -XX:+CrashOnOutOfMemoryError, or -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\"");
+            }
+            else
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError="))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                            + " Either upgrade your JRE to a version greater or equal to 8u92 and use -XX:+ExitOnOutOfMemoryError/-XX:+CrashOnOutOfMemoryError"
+                            + " or use -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\" on your current JRE.");
+            }
+        }
+
+        /**
+         * Returns the java version number for an Oracle JVM.
+         * @return the java version number
+         */
+        private int getJavaVersion()
+        {
+            String jreVersion = System.getProperty("java.version");
+            String version = jreVersion.startsWith("1.") ? jreVersion.substring(2, 3) // Pre 9 version
+                                                         : jreVersion.substring(0, jreVersion.indexOf('.'));
+            return Integer.parseInt(version);
+        }
+
+        /**
+         * Return the update number for an Oracle JVM.
+         * @return the update number
+         */
+        private int getUpdate()
+        {
+            String jreVersion = System.getProperty("java.version");
+            int updateSeparatorIndex = jreVersion.indexOf('_');
+            return Integer.parseInt(jreVersion.substring(updateSeparatorIndex + 1));
+        }
+
+        /**
+         * Checks if one of the specified options is being used.
+         * @param optionNames The name of the options to check
+         * @return {@code true} if one of the specified options is being used, {@code false} otherwise.
+         */
+        private boolean jvmOptionsContainsOneOf(String... optionNames)
+        {
+            RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
+            List<String> inputArguments = runtimeMxBean.getInputArguments();
+            for (String argument : inputArguments)
+            {
+                for (String optionName : optionNames)
+                    if (argument.startsWith(optionName))
+                        return true;
+            }
+            return false;
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 bfc8a0b..2d068de 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -19,11 +19,6 @@ package org.apache.cassandra.utils;
 
 import java.io.*;
 import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.text.StrBuilder;
@@ -32,7 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Utility to generate heap dumps.
+ * Utility to log heap histogram.
  *
  */
 public final class HeapUtils
@@ -43,54 +38,33 @@ public final class HeapUtils
      * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
      * or in the <code>CASSANDRA_HOME</code> directory.
      */
-    public static void generateHeapDump()
+    public static void logHeapHistogram()
     {
-        Long processId = getProcessId();
-        if (processId == null)
+        try
         {
-            logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-            return;
-        }
+            logger.info("Trying to log the heap histogram using jmap");
 
-        String heapDumpPath = getHeapDumpPathOption();
-        if (heapDumpPath == null)
-        {
-            String cassandraHome = System.getenv("CASSANDRA_HOME");
-            if (cassandraHome == null)
+            Long processId = getProcessId();
+            if (processId == null)
             {
+                logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                 return;
             }
 
-            heapDumpPath = cassandraHome;
-        }
+            String jmapPath = getJmapPath();
 
-        Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-        if (Files.isDirectory(dumpPath))
-        {
-            dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-        }
+            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
+            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
 
-        String jmapPath = getJmapPath();
+            String[] histoCommands = new String[] {jmapCommand,
+                    "-histo",
+                    processId.toString()};
 
-        // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-        String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
-
-        String[] dumpCommands = new String[] {jmapCommand,
-                                              "-dump:format=b,file=" + dumpPath,
-                                              processId.toString()};
-
-        // Lets also log the Heap histogram
-        String[] histoCommands = new String[] {jmapCommand,
-                                               "-histo",
-                                               processId.toString()};
-        try
-        {
-            logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
             logProcessOutput(Runtime.getRuntime().exec(histoCommands));
         }
-        catch (IOException e)
+        catch (Throwable e)
         {
-            logger.error("The heap dump could not be generated due to the following error: ", e);
+            logger.error("The heap histogram could not be generated due to the following error: ", e);
         }
     }
 
@@ -137,32 +111,6 @@ public final class HeapUtils
     }
 
     /**
-     * Retrieves the value of the <code>HeapDumpPath</code> JVM option.
-     * @return the value of the <code>HeapDumpPath</code> JVM option or <code>null</code> if the value has not been
-     * specified.
-     */
-    private static String getHeapDumpPathOption()
-    {
-        RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
-        List<String> inputArguments = runtimeMxBean.getInputArguments();
-        String heapDumpPathOption = null;
-        for (String argument : inputArguments)
-        {
-            if (argument.startsWith("-XX:HeapDumpPath="))
-            {
-                heapDumpPathOption = argument;
-                // We do not break in case the option has been specified several times.
-                // In general it seems that JVMs use the right-most argument as the winner.
-            }
-        }
-
-        if (heapDumpPathOption == null)
-            return null;
-
-        return heapDumpPathOption.substring(17, heapDumpPathOption.length());
-    }
-
-    /**
      * Retrieves the process ID or <code>null</code> if the process ID cannot be retrieved.
      * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index f8cb775..0196b04 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.net.SocketException;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,6 +39,8 @@ public final class JVMStabilityInspector
     private static final Logger logger = LoggerFactory.getLogger(JVMStabilityInspector.class);
     private static Killer killer = new Killer();
 
+    private static Object lock = new Object();
+    private static boolean printingHeapHistogram;
 
     private JVMStabilityInspector() {}
 
@@ -52,8 +55,25 @@ public final class JVMStabilityInspector
         boolean isUnstable = false;
         if (t instanceof OutOfMemoryError)
         {
-            isUnstable = true;
-            HeapUtils.generateHeapDump();
+            if (Boolean.getBoolean("cassandra.printHeapHistogramOnOutOfMemoryError"))
+            {
+                // We want to avoid printing multiple time the heap histogram if multiple OOM errors happen in a short
+                // time span.
+                synchronized(lock)
+                {
+                    if (printingHeapHistogram)
+                        return;
+                    printingHeapHistogram = true;
+                }
+                HeapUtils.logHeapHistogram();
+            }
+
+            logger.error("OutOfMemory error letting the JVM handle the error:", t);
+
+            StorageService.instance.removeShutdownHook();
+            // We let the JVM handle the error. The startup checks should have warned the user if it did not configure
+            // the JVM behavior in case of OOM (CASSANDRA-13006).
+            throw (OutOfMemoryError) t;
         }
 
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index 7142f97..f96ac6e 100644
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@ -20,14 +20,19 @@ package org.apache.cassandra.utils;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+
+import static java.util.Arrays.asList;
+
 import org.junit.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.SocketException;
+import java.util.Arrays;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class JVMStabilityInspectorTest
 {
@@ -45,10 +50,6 @@ public class JVMStabilityInspectorTest
             JVMStabilityInspector.inspectThrowable(new IOException());
             assertFalse(killerForTests.wasKilled());
 
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new OutOfMemoryError());
-            assertTrue(killerForTests.wasKilled());
-
             DatabaseDescriptor.setDiskFailurePolicy(Config.DiskFailurePolicy.die);
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new FSReadError(new IOException(), "blah"));
@@ -62,11 +63,6 @@ public class JVMStabilityInspectorTest
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new Exception(new IOException()));
             assertFalse(killerForTests.wasKilled());
-
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new Exception(new OutOfMemoryError()));
-            assertTrue(killerForTests.wasKilled());
-
         }
         finally
         {
@@ -77,6 +73,23 @@ public class JVMStabilityInspectorTest
     }
 
     @Test
+    public void testOutOfMemoryHandling()
+    {
+        for (Throwable oom : asList(new OutOfMemoryError(), new Exception(new OutOfMemoryError())))
+        {
+            try
+            {
+                JVMStabilityInspector.inspectThrowable(oom);
+                fail("The JVMStabilityInspector should delegate the handling of OutOfMemoryErrors to the JVM");
+            }
+            catch (OutOfMemoryError e)
+            {
+                assertTrue(true);
+            }
+        }
+    }
+
+    @Test
     public void fileHandleTest()
     {
         KillerForTests killerForTests = new KillerForTests();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[05/10] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
Merge branch cassandra-2.2 into cassandra-3.0


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

Branch: refs/heads/cassandra-3.11
Commit: dd187d105bce300395f52eb8ca4d8ccaf94a06ea
Parents: d7329a6 02aba73
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:30:29 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:32:36 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +-
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 950 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 650 insertions(+), 560 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 8cff716,5200eb1..20ccc4b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,83 -1,8 +1,84 @@@
 -2.2.12
 +3.0.16
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
+  * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
   * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  
 -2.2.11
 +
 +3.0.15
 + * Improve TRUNCATE performance (CASSANDRA-13909)
 + * Implement short read protection on partition boundaries (CASSANDRA-13595)
 + * Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries (CASSANDRA-13911)
 + * Filter header only commit logs before recovery (CASSANDRA-13918)
 + * AssertionError prepending to a list (CASSANDRA-13149)
 + * Fix support for SuperColumn tables (CASSANDRA-12373)
 + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883)
 + * Fix missing original update in TriggerExecutor (CASSANDRA-13894)
 + * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043)
 + * Improve short read protection performance (CASSANDRA-13794)
 + * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787)
 + * Fix short read protection for tables with no clustering columns (CASSANDRA-13880)
 + * Make isBuilt volatile in PartitionUpdate (CASSANDRA-13619)
 + * Prevent integer overflow of timestamps in CellTest and RowsTest (CASSANDRA-13866)
 + * Fix counter application order in short read protection (CASSANDRA-12872)
 + * Don't block RepairJob execution on validation futures (CASSANDRA-13797)
 + * Wait for all management tasks to complete before shutting down CLSM (CASSANDRA-13123)
 + * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
 + * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
 + * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
 + * Range deletes in a CAS batch are ignored (CASSANDRA-13655)
 + * Avoid assertion error when IndexSummary > 2G (CASSANDRA-12014)
 + * Change repair midpoint logging for tiny ranges (CASSANDRA-13603)
 + * Better handle corrupt final commitlog segment (CASSANDRA-11995)
 + * StreamingHistogram is not thread safe (CASSANDRA-13756)
 + * Fix MV timestamp issues (CASSANDRA-11500)
 + * Better tolerate improperly formatted bcrypt hashes (CASSANDRA-13626) 
 + * Fix race condition in read command serialization (CASSANDRA-13363)
 + * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
 + * Fix AssertionError in short read protection (CASSANDRA-13747)
 + * Don't skip corrupted sstables on startup (CASSANDRA-13620)
 + * Fix the merging of cells with different user type versions (CASSANDRA-13776)
 + * Copy session properties on cqlsh.py do_login (CASSANDRA-13640)
 + * Potential AssertionError during ReadRepair of range tombstone and partition deletions (CASSANDRA-13719)
 + * Don't let stress write warmup data if n=0 (CASSANDRA-13773)
 + * Gossip thread slows down when using batch commit log (CASSANDRA-12966)
 + * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
 + * Fix digest calculation for counter cells (CASSANDRA-13750)
 + * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
 + * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)
 + * Drop table should remove corresponding entries in dropped_columns table (CASSANDRA-13730)
 + * Log warn message until legacy auth tables have been migrated (CASSANDRA-13371)
 + * Fix incorrect [2.1 <- 3.0] serialization of counter cells created in 2.0 (CASSANDRA-13691)
 + * Fix invalid writetime for null cells (CASSANDRA-13711)
 + * Fix ALTER TABLE statement to atomically propagate changes to the table and its MVs (CASSANDRA-12952)
 + * Fixed ambiguous output of nodetool tablestats command (CASSANDRA-13722)
 + * JMXEnabledThreadPoolExecutor with corePoolSize equal to maxPoolSize (Backport CASSANDRA-13329)
 + * Fix Digest mismatch Exception if hints file has UnknownColumnFamily (CASSANDRA-13696)
 + * Purge tombstones created by expired cells (CASSANDRA-13643)
 + * Make concat work with iterators that have different subsets of columns (CASSANDRA-13482)
 + * Set test.runners based on cores and memory size (CASSANDRA-13078)
 + * Allow different NUMACTL_ARGS to be passed in (CASSANDRA-13557)
 + * Allow native function calls in CQLSSTableWriter (CASSANDRA-12606)
 + * Fix secondary index queries on COMPACT tables (CASSANDRA-13627)
 + * Nodetool listsnapshots output is missing a newline, if there are no snapshots (CASSANDRA-13568)
 + * sstabledump reports incorrect usage for argument order (CASSANDRA-13532)
 +Merged from 2.2:
   * Safely handle empty buffers when outputting to JSON (CASSANDRA-13868)
   * Copy session properties on cqlsh.py do_login (CASSANDRA-13847)
   * Fix load over calculated issue in IndexSummaryRedistribution (CASSANDRA-13738)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 621866b,5747941..1bbe1aa
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,66 -13,7 +13,71 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
-    - Nothing specific to this release, but please see previous upgrading sections,
-      especially if you are upgrading from 2.2.
++   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
++     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
++     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
++     options are used. See CASSANDRA-13006 for more details.
++   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
++     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
++     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/bin/cassandra
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index d3a0a87,7b4a632..405ed92
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -1,480 -1,482 +1,490 @@@
--#
--# 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.
--
--# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
--
--#-----------------------------------------------------------------------------
--Function SetCassandraHome()
--{
--    if (! $env:CASSANDRA_HOME)
--    {
--        $cwd = [System.IO.Directory]::GetCurrentDirectory()
--        $cwd = Split-Path $cwd -parent
--        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraMain()
--{
--    if (! $env:CASSANDRA_MAIN)
--    {
--        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function BuildClassPath
--{
--    $cp = """$env:CASSANDRA_HOME\conf"""
--    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
--    {
--        $file = $file -replace "\\", "/"
--        $cp = $cp + ";" + """$file"""
--    }
--
--    # Add build/classes/main so it works in development
--    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
--    $env:CLASSPATH=$cp
--}
--
--#-----------------------------------------------------------------------------
--Function CalculateHeapSizes
--{
--    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
--
--    $osInfo = Get-WmiObject -class "Win32_computersystem"
--    $autoPage = $osInfo.AutomaticManagedPageFile
--
--    if ($autoPage)
--    {
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--        echo ""
--        echo "    WARNING!  Automatic page file configuration detected."
--        echo "    It is recommended that you disable swap when running Cassandra"
--        echo "    for performance and stability reasons."
--        echo ""
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--    }
--    else
--    {
--        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
--        $pageFileCount = $PageFileInfo.Count
--        if ($pageFileInfo)
--        {
--            $files = @()
--            $sizes = @()
--            $hasSizes = $FALSE
--
--            # PageFileCount isn't populated and obj comes back as single if there's only 1
--            if ([string]::IsNullOrEmpty($PageFileCount))
--            {
--                $PageFileCount = 1
--                $files += $PageFileInfo.Name
--                if ($PageFileInfo.MaximumSize -ne 0)
--                {
--                    $hasSizes = $TRUE
--                    $sizes += $PageFileInfo.MaximumSize
--                }
--            }
--            else
--            {
--                for ($i = 0; $i -le $PageFileCount; $i++)
--                {
--                    $files += $PageFileInfo[$i].Name
--                    if ($PageFileInfo[$i].MaximumSize -ne 0)
--                    {
--                        $hasSizes = $TRUE
--                        $sizes += $PageFileInfo[$i].MaximumSize
--                    }
--                }
--            }
--
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING!  $PageFileCount swap file(s) detected"
--            for ($i = 0; $i -lt $PageFileCount; $i++)
--            {
--                $toPrint = "        Name: " + $files[$i]
--                if ($hasSizes)
--                {
--                    $toPrint = $toPrint + " Size: " + $sizes[$i]
--                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
--                }
--                echo $toPrint
--            }
--            echo "    It is recommended that you disable swap when running Cassandra"
--            echo "    for performance and stability reasons."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
--    }
--
--    # Validate that we need to run this function and that our config is good
--    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
--    {
--        return
--    }
- 
-     if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
 -    if (($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE))
--    {
--        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
--        exit 1
--    }
--
--    $memObject = Get-WMIObject -class win32_physicalmemory
--    if ($memObject -eq $null)
--    {
-         echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
 -        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\cassandra-env.ps1 for different heap values."
--        $env:MAX_HEAP_SIZE = "2048M"
--        $env:HEAP_NEWSIZE = "512M"
--        return
--    }
--
--    $memory = ($memObject | Measure-Object Capacity -Sum).sum
--    $memoryMB = [Math]::Truncate($memory / (1024*1024))
--
--    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
--    $systemCores = $cpu.NumberOfLogicalProcessors
--
--    # set max heap size based on the following
--    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
--    # calculate 1/2 ram and cap to 1024MB
--    # calculate 1/4 ram and cap to 8192MB
--    # pick the max
--    $halfMem = [Math]::Truncate($memoryMB / 2)
--    $quarterMem = [Math]::Truncate($halfMem / 2)
--
--    if ($halfMem -gt 1024)
--    {
--        $halfMem = 1024
--    }
--    if ($quarterMem -gt 8192)
--    {
--        $quarterMem = 8192
--    }
--
--    $maxHeapMB = ""
--    if ($halfMem -gt $quarterMem)
--    {
--        $maxHeapMB = $halfMem
--    }
--    else
--    {
--        $maxHeapMB = $quarterMem
--    }
--    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
--
--    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
--    $maxYGPerCore = 100
--    $maxYGTotal = $maxYGPerCore * $systemCores
--    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
--
--    if ($desiredYG -gt $maxYGTotal)
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
--    }
--    else
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
 -    }
 -}
 -
 -#-----------------------------------------------------------------------------
 -Function SetJsr223Env
 -{
 -    $cp = $env:CLASSPATH
 -    foreach ($jsrDir in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223")
 -    {
 -        foreach ($file in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223\$jsrDir\*.jar")
 -        {
 -            $file = $file -replace "\\", "/"
 -			$cp = $cp + ";" + """$file"""
 -        }
--    }
 -    $env:CLASSPATH=$cp
 -
 -	# JSR223/JRuby - set ruby lib directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\ruby")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djruby.lib=$env:CASSANDRA_HOME\lib\jsr223\jruby"
 -	}
 -	# JSR223/JRuby - set ruby JNI libraries root directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\jni")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djffi.boot.library.path=$env:CASSANDRA_HOME\lib\jsr223\jruby\jni"
 -	}
 -	# JSR223/Jython - set python.home system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jython\jython.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dpython.home=$env:CASSANDRA_HOME\lib\jsr223\jython"
 -	}
 -	# JSR223/Scala - necessary system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\scala\scala-compiler.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dscala.usejavacp=true"
 -	}
--}
--
--#-----------------------------------------------------------------------------
--Function ParseJVMInfo
--{
--    # grab info about the JVM
--    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
--    $pinfo.FileName = "$env:JAVA_BIN"
--    $pinfo.RedirectStandardError = $true
--    $pinfo.RedirectStandardOutput = $true
--    $pinfo.UseShellExecute = $false
--    $pinfo.Arguments = "-d64 -version"
--    $p = New-Object System.Diagnostics.Process
--    $p.StartInfo = $pinfo
--    $p.Start() | Out-Null
--    $p.WaitForExit()
--    $stderr = $p.StandardError.ReadToEnd()
--
--    $env:JVM_ARCH = "64-bit"
--
--    if ($stderr.Contains("Error"))
--    {
--        # 32-bit JVM. re-run w/out -d64
--        echo "Failed 64-bit check. Re-running to get version from 32-bit"
--        $pinfo.Arguments = "-version"
--        $p = New-Object System.Diagnostics.Process
--        $p.StartInfo = $pinfo
--        $p.Start() | Out-Null
--        $p.WaitForExit()
--        $stderr = $p.StandardError.ReadToEnd()
--        $env:JVM_ARCH = "32-bit"
--    }
--
--    $sa = $stderr.Split("""")
--    $env:JVM_VERSION = $sa[1]
--
--    if ($stderr.Contains("OpenJDK"))
--    {
--        $env:JVM_VENDOR = "OpenJDK"
--    }
--    elseif ($stderr.Contains("Java(TM)"))
--    {
--        $env:JVM_VENDOR = "Oracle"
--    }
--    else
--    {
--        $JVM_VENDOR = "other"
--    }
--
--    $pa = $sa[1].Split("_")
-     $subVersion = $pa[1]
-     # Deal with -b (build) versions
-     if ($subVersion -contains '-')
-     {
-         $patchAndBuild = $subVersion.Split("-")
-         $subVersion = $patchAndBuild[0]
-     }
-     $env:JVM_PATCH_VERSION = $subVersion
 -    $env:JVM_PATCH_VERSION=$pa[1]
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraEnvironment
--{
--    if (Test-Path Env:\JAVA_HOME)
--    {
--        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
--    }
--    elseif (Get-Command "java.exe")
--    {
--        $env:JAVA_BIN = "java.exe"
--    }
--    else
--    {
--        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
--        exit
--    }
--    SetCassandraHome
--    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
--    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
--
--    $logdir = "$env:CASSANDRA_HOME\logs"
--    $storagedir = "$env:CASSANDRA_HOME\data"
--    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
--
--    SetCassandraMain
--    BuildClassPath
 -    SetJsr223Env
--
--    # Override these to set the amount of memory to allocate to the JVM at
--    # start-up. For production use you may wish to adjust this for your
--    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
-     # to the Java heap. HEAP_NEWSIZE refers to the size of the young
 -    # to the Java heap; HEAP_NEWSIZE refers to the size of the young
--    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
--    # or not (if you set one, set the other).
--    #
--    # The main trade-off for the young generation is that the larger it
--    # is, the longer GC pause times will be. The shorter it is, the more
--    # expensive GC will be (usually).
--    #
--    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
--    # times. If in doubt, and if you do not particularly want to tweak, go
--    # 100 MB per physical CPU core.
- 
-     #GC log path has to be defined here since it needs to find CASSANDRA_HOME
-     $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
- 
-     # Read user-defined JVM options from jvm.options file
-     $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
-     for ($i = 0; $i -lt $content.Count; $i++)
-     {
-         $line = $content[$i]
-         if ($line.StartsWith("-"))
-         {
-             $env:JVM_OPTS = "$env:JVM_OPTS $line"
-         }
-     }
- 
-     $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
-     $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
-     $defined_xms = $env:JVM_OPTS -like '*Xms*'
-     $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
--
--    #$env:MAX_HEAP_SIZE="4096M"
--    #$env:HEAP_NEWSIZE="800M"
--    CalculateHeapSizes
--
--    ParseJVMInfo
- 
-     # We only set -Xms and -Xmx if they were not defined on jvm.options file
-     # If defined, both Xmx and Xms should be defined together.
-     if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
-     }
-     elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
-         exit
-     }
- 
-     # We only set -Xmn flag if it was not defined in jvm.options file
-     # and if the CMS GC is being used
-     # If defined, both Xmn and Xmx should be defined together.
-     if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
-         exit
-     }
-     elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
-     }
- 
-     if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
-     }
- 
--    # Add sigar env - see Cassandra-7838
--    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
--
--    # Confirm we're on high performance power plan, warn if not
--    # Change to $true to suppress this warning
--    $suppressPowerWarning = $false
--    if (!$suppressPowerWarning)
--    {
--        $currentProfile = powercfg /GETACTIVESCHEME
--        if (!$currentProfile.Contains("High performance"))
--        {
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING! Detected a power profile other than High Performance."
--            echo "    Performance of this node will suffer."
--            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
 -    }
 -
 -    # add the jamm javaagent
 -    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
 -        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
 -    {
 -        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
 -    }
 -
 -    # enable assertions.  disabling this in production will give a modest
 -    # performance benefit (around 5%).
 -    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
 -
 -    # Specifies the default port over which Cassandra will be available for
 -    # JMX connections.
 -    $JMX_PORT="7199"
 -
 -    # store in env to check if it's avail in verification
 -    $env:JMX_PORT=$JMX_PORT
 -
 -    # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled"
 -
 -    # enable thread priorities, primarily so we can give periodic tasks
 -    # a lower priority to avoid interfering with client workload
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
 -    # allows lowering thread priority without being root on linux - probably
 -    # not necessary on Windows but doesn't harm anything.
 -    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
 -
 -    # min and max heap sizes should be set to the same value to avoid
 -    # stop-the-world GC pauses during resize.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 -
 -    # stop the jvm on OutOfMemoryError as it can result in some data corruption
 -    # uncomment the preferred option
 -    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
 -
 -    # print an heap histogram on OutOfMemoryError
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
 -
 -    # Per-thread stack size.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 -
 -    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
 -
 -    # GC tuning options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseParNewGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseConcMarkSweepGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelRemarkEnabled"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:SurvivorRatio=8"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:MaxTenuringThreshold=1"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB"
 -    if (($env:JVM_VERSION.CompareTo("1.7") -eq 1) -and ($env:JVM_ARCH -eq "64-Bit"))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
 -    }
 -    if ( (($env:JVM_VERSION.CompareTo("1.7") -ge 0) -and ($env:JVM_PATCH_VERSION.CompareTo("60") -ge 0)) -or
 -         ($env:JVM_VERSION.CompareTo("1.8") -ge 0))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways"
--    }
 -
 -    # GC logging options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDetails"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDateStamps"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintHeapAtGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintTenuringDistribution"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintPromotionFailure"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseGCLogFileRotation"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:GCLogFileSize=10M"
 -    # if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -    # $currentDate = (Get-Date).ToString('yyyy.MM.dd')
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:$env:CASSANDRA_HOME/logs/gc-$currentDate.log"
 -
 -    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
 -    # library path.
 -    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
 -
 -    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -    # comment out this entry to enable IPv6 support).
 -    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
 -    # jmx: metrics and administration interface
 -    #
 -    # add this if you're having trouble connecting:
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
 -    #
 -    # see
 -    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
 -    # for more on configuring JMX through firewalls, etc. (Short version:
 -    # get it working with no firewall first.)
 -    #
 -    # Due to potential security exploits, Cassandra ships with JMX accessible
 -    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
 -    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity 
 -    #
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
--
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 -}
++#
++# 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.
++
++# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
++
++#-----------------------------------------------------------------------------
++Function SetCassandraHome()
++{
++    if (! $env:CASSANDRA_HOME)
++    {
++        $cwd = [System.IO.Directory]::GetCurrentDirectory()
++        $cwd = Split-Path $cwd -parent
++        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraMain()
++{
++    if (! $env:CASSANDRA_MAIN)
++    {
++        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function BuildClassPath
++{
++    $cp = """$env:CASSANDRA_HOME\conf"""
++    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
++    {
++        $file = $file -replace "\\", "/"
++        $cp = $cp + ";" + """$file"""
++    }
++
++    # Add build/classes/main so it works in development
++    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
++    $env:CLASSPATH=$cp
++}
++
++#-----------------------------------------------------------------------------
++Function CalculateHeapSizes
++{
++    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
++
++    $osInfo = Get-WmiObject -class "Win32_computersystem"
++    $autoPage = $osInfo.AutomaticManagedPageFile
++
++    if ($autoPage)
++    {
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++        echo ""
++        echo "    WARNING!  Automatic page file configuration detected."
++        echo "    It is recommended that you disable swap when running Cassandra"
++        echo "    for performance and stability reasons."
++        echo ""
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++    }
++    else
++    {
++        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
++        $pageFileCount = $PageFileInfo.Count
++        if ($pageFileInfo)
++        {
++            $files = @()
++            $sizes = @()
++            $hasSizes = $FALSE
++
++            # PageFileCount isn't populated and obj comes back as single if there's only 1
++            if ([string]::IsNullOrEmpty($PageFileCount))
++            {
++                $PageFileCount = 1
++                $files += $PageFileInfo.Name
++                if ($PageFileInfo.MaximumSize -ne 0)
++                {
++                    $hasSizes = $TRUE
++                    $sizes += $PageFileInfo.MaximumSize
++                }
++            }
++            else
++            {
++                for ($i = 0; $i -le $PageFileCount; $i++)
++                {
++                    $files += $PageFileInfo[$i].Name
++                    if ($PageFileInfo[$i].MaximumSize -ne 0)
++                    {
++                        $hasSizes = $TRUE
++                        $sizes += $PageFileInfo[$i].MaximumSize
++                    }
++                }
++            }
++
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING!  $PageFileCount swap file(s) detected"
++            for ($i = 0; $i -lt $PageFileCount; $i++)
++            {
++                $toPrint = "        Name: " + $files[$i]
++                if ($hasSizes)
++                {
++                    $toPrint = $toPrint + " Size: " + $sizes[$i]
++                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
++                }
++                echo $toPrint
++            }
++            echo "    It is recommended that you disable swap when running Cassandra"
++            echo "    for performance and stability reasons."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
++    # Validate that we need to run this function and that our config is good
++    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
++    {
++        return
++    }
++
++    if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
++    {
++        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
++        exit 1
++    }
++
++    $memObject = Get-WMIObject -class win32_physicalmemory
++    if ($memObject -eq $null)
++    {
++        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
++        $env:MAX_HEAP_SIZE = "2048M"
++        $env:HEAP_NEWSIZE = "512M"
++        return
++    }
++
++    $memory = ($memObject | Measure-Object Capacity -Sum).sum
++    $memoryMB = [Math]::Truncate($memory / (1024*1024))
++
++    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
++    $systemCores = $cpu.NumberOfLogicalProcessors
++
++    # set max heap size based on the following
++    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
++    # calculate 1/2 ram and cap to 1024MB
++    # calculate 1/4 ram and cap to 8192MB
++    # pick the max
++    $halfMem = [Math]::Truncate($memoryMB / 2)
++    $quarterMem = [Math]::Truncate($halfMem / 2)
++
++    if ($halfMem -gt 1024)
++    {
++        $halfMem = 1024
++    }
++    if ($quarterMem -gt 8192)
++    {
++        $quarterMem = 8192
++    }
++
++    $maxHeapMB = ""
++    if ($halfMem -gt $quarterMem)
++    {
++        $maxHeapMB = $halfMem
++    }
++    else
++    {
++        $maxHeapMB = $quarterMem
++    }
++    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
++
++    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
++    $maxYGPerCore = 100
++    $maxYGTotal = $maxYGPerCore * $systemCores
++    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
++
++    if ($desiredYG -gt $maxYGTotal)
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
++    }
++    else
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function ParseJVMInfo
++{
++    # grab info about the JVM
++    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
++    $pinfo.FileName = "$env:JAVA_BIN"
++    $pinfo.RedirectStandardError = $true
++    $pinfo.RedirectStandardOutput = $true
++    $pinfo.UseShellExecute = $false
++    $pinfo.Arguments = "-d64 -version"
++    $p = New-Object System.Diagnostics.Process
++    $p.StartInfo = $pinfo
++    $p.Start() | Out-Null
++    $p.WaitForExit()
++    $stderr = $p.StandardError.ReadToEnd()
++
++    $env:JVM_ARCH = "64-bit"
++
++    if ($stderr.Contains("Error"))
++    {
++        # 32-bit JVM. re-run w/out -d64
++        echo "Failed 64-bit check. Re-running to get version from 32-bit"
++        $pinfo.Arguments = "-version"
++        $p = New-Object System.Diagnostics.Process
++        $p.StartInfo = $pinfo
++        $p.Start() | Out-Null
++        $p.WaitForExit()
++        $stderr = $p.StandardError.ReadToEnd()
++        $env:JVM_ARCH = "32-bit"
++    }
++
++    $sa = $stderr.Split("""")
++    $env:JVM_VERSION = $sa[1]
++
++    if ($stderr.Contains("OpenJDK"))
++    {
++        $env:JVM_VENDOR = "OpenJDK"
++    }
++    elseif ($stderr.Contains("Java(TM)"))
++    {
++        $env:JVM_VENDOR = "Oracle"
++    }
++    else
++    {
++        $JVM_VENDOR = "other"
++    }
++
++    $pa = $sa[1].Split("_")
++    $subVersion = $pa[1]
++    # Deal with -b (build) versions
++    if ($subVersion -contains '-')
++    {
++        $patchAndBuild = $subVersion.Split("-")
++        $subVersion = $patchAndBuild[0]
++    }
++    $env:JVM_PATCH_VERSION = $subVersion
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraEnvironment
++{
++    if (Test-Path Env:\JAVA_HOME)
++    {
++        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
++    }
++    elseif (Get-Command "java.exe")
++    {
++        $env:JAVA_BIN = "java.exe"
++    }
++    else
++    {
++        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
++        exit
++    }
++    SetCassandraHome
++    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
++    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
++
++    $logdir = "$env:CASSANDRA_HOME\logs"
++    $storagedir = "$env:CASSANDRA_HOME\data"
++    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
++
++    SetCassandraMain
++    BuildClassPath
++
++    # Override these to set the amount of memory to allocate to the JVM at
++    # start-up. For production use you may wish to adjust this for your
++    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
++    # to the Java heap. HEAP_NEWSIZE refers to the size of the young
++    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
++    # or not (if you set one, set the other).
++    #
++    # The main trade-off for the young generation is that the larger it
++    # is, the longer GC pause times will be. The shorter it is, the more
++    # expensive GC will be (usually).
++    #
++    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
++    # times. If in doubt, and if you do not particularly want to tweak, go
++    # 100 MB per physical CPU core.
++
++    #GC log path has to be defined here since it needs to find CASSANDRA_HOME
++    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
++
++    # Read user-defined JVM options from jvm.options file
++    $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
++    for ($i = 0; $i -lt $content.Count; $i++)
++    {
++        $line = $content[$i]
++        if ($line.StartsWith("-"))
++        {
++            $env:JVM_OPTS = "$env:JVM_OPTS $line"
++        }
++    }
++
++    $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
++    $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
++    $defined_xms = $env:JVM_OPTS -like '*Xms*'
++    $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
++
++    #$env:MAX_HEAP_SIZE="4096M"
++    #$env:HEAP_NEWSIZE="800M"
++    CalculateHeapSizes
++
++    ParseJVMInfo
++
++    # We only set -Xms and -Xmx if they were not defined on jvm.options file
++    # If defined, both Xmx and Xms should be defined together.
++    if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
++    }
++    elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
++        exit
++    }
++
++    # We only set -Xmn flag if it was not defined in jvm.options file
++    # and if the CMS GC is being used
++    # If defined, both Xmn and Xmx should be defined together.
++    if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
++        exit
++    }
++    elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
++    }
++
++    if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
++    }
++
++    # Add sigar env - see Cassandra-7838
++    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
++
++    # Confirm we're on high performance power plan, warn if not
++    # Change to $true to suppress this warning
++    $suppressPowerWarning = $false
++    if (!$suppressPowerWarning)
++    {
++        $currentProfile = powercfg /GETACTIVESCHEME
++        if (!$currentProfile.Contains("High performance"))
++        {
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING! Detected a power profile other than High Performance."
++            echo "    Performance of this node will suffer."
++            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
 +    # provides hints to the JIT compiler
 +    $env:JVM_OPTS = "$env:JVM_OPTS -XX:CompileCommandFile=$env:CASSANDRA_CONF\hotspot_compiler"
 +
-     # add the jamm javaagent
-     if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
-         (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
-     {
-         $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
-     }
- 
++    # add the jamm javaagent
++    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
++        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
++    {
++        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
++    }
++
 +    # set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR
 +    if ($env:CASSANDRA_HEAPDUMP_DIR)
 +    {
 +        $unixTimestamp = [int64](([datetime]::UtcNow)-(get-date "1/1/1970")).TotalSeconds
 +        $env:JVM_OPTS="$env:JVM_OPTS -XX:HeapDumpPath=$env:CASSANDRA_HEAPDUMP_DIR\cassandra-$unixTimestamp-pid$pid.hprof"
 +    }
 +
-     if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
-     {
-         echo "Cassandra 3.0 and later require Java 8u40 or later."
-         exit
-     }
- 
-     # enable assertions.  disabling this in production will give a modest
-     # performance benefit (around 5%).
-     $env:JVM_OPTS = "$env:JVM_OPTS -ea"
- 
-     # Specifies the default port over which Cassandra will be available for
-     # JMX connections.
-     $JMX_PORT="7199"
- 
-     # store in env to check if it's avail in verification
-     $env:JMX_PORT=$JMX_PORT
- 
-     # enable thread priorities, primarily so we can give periodic tasks
-     # a lower priority to avoid interfering with client workload
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
-     # allows lowering thread priority without being root on linux - probably
-     # not necessary on Windows but doesn't harm anything.
-     # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
- 
-     # Per-thread stack size.
-     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
- 
-     # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
- 
-     # Make sure all memory is faulted and zeroed on startup.
-     # This helps prevent soft faults in containers and makes
-     # transparent hugepage allocation more effective.
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
- 
-     # Biased locking does not benefit Cassandra.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
- 
-     # Enable thread-local allocation blocks and allow the JVM to automatically
-     # resize them at runtime.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
- 
-     # http://www.evanjones.ca/jvm-mmap-pause.html
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
- 
-     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
-     # library path.
-     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
- 
-     # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
-     # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
- 
-     # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
-     # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
-     # comment out this entry to enable IPv6 support).
-     $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
- 
-     # jmx: metrics and administration interface
-     #
-     # add this if you're having trouble connecting:
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
-     #
-     # see
-     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
-     # for more on configuring JMX through firewalls, etc. (Short version:
-     # get it working with no firewall first.)
-     #
-     # Due to potential security exploits, Cassandra ships with JMX accessible
-     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
-     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
-     #
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-     $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
- 
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
- }
++    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
++    {
++        echo "Cassandra 3.0 and later require Java 8u40 or later."
++        exit
++    }
++
++    # enable assertions.  disabling this in production will give a modest
++    # performance benefit (around 5%).
++    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
++
++    # Specifies the default port over which Cassandra will be available for
++    # JMX connections.
++    $JMX_PORT="7199"
++
++    # store in env to check if it's avail in verification
++    $env:JMX_PORT=$JMX_PORT
++
++    # enable thread priorities, primarily so we can give periodic tasks
++    # a lower priority to avoid interfering with client workload
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
++    # allows lowering thread priority without being root on linux - probably
++    # not necessary on Windows but doesn't harm anything.
++    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
++
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
++
++    # stop the jvm on OutOfMemoryError as it can result in some data corruption
++    # uncomment the preferred option
++    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
++
++    # print an heap histogram on OutOfMemoryError
++    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
++
++    # Per-thread stack size.
++    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
++
++    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
++
++    # Make sure all memory is faulted and zeroed on startup.
++    # This helps prevent soft faults in containers and makes
++    # transparent hugepage allocation more effective.
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
++
++    # Biased locking does not benefit Cassandra.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
++
++    # Enable thread-local allocation blocks and allow the JVM to automatically
++    # resize them at runtime.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
++
++    # http://www.evanjones.ca/jvm-mmap-pause.html
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
++
++    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
++    # library path.
++    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
++
++    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
++    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
++
++    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
++    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
++    # comment out this entry to enable IPv6 support).
++    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
++
++    # jmx: metrics and administration interface
++    #
++    # add this if you're having trouble connecting:
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
++    #
++    # see
++    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
++    # for more on configuring JMX through firewalls, etc. (Short version:
++    # get it working with no firewall first.)
++    #
++    # Due to potential security exploits, Cassandra ships with JMX accessible
++    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
++    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
++    #
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
++    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
++
++    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
++
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --cc conf/cassandra-env.sh
index 44fe110,7b1b8d3..ffc773f
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@@ -247,6 -201,65 +247,18 @@@ if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" 
      JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
  fi
  
 -
 -startswith() { [ "${1#$2}" != "$1" ]; }
 -
+ # stop the jvm on OutOfMemoryError as it can result in some data corruption
+ # uncomment the preferred option
++# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+ # on white spaces without taking quotes into account
 -# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+ # JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+ JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+ 
+ # print an heap histogram on OutOfMemoryError
+ # JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -# Per-thread stack size.
 -JVM_OPTS="$JVM_OPTS -Xss256k"
 -
 -# Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -JVM_OPTS="$JVM_OPTS -XX:StringTableSize=1000003"
 -
 -# GC tuning options
 -JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+UseConcMarkSweepGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+CMSParallelRemarkEnabled" 
 -JVM_OPTS="$JVM_OPTS -XX:SurvivorRatio=8" 
 -JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=1"
 -JVM_OPTS="$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -JVM_OPTS="$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -JVM_OPTS="$JVM_OPTS -XX:+UseTLAB"
 -JVM_OPTS="$JVM_OPTS -XX:+PerfDisableSharedMem"
 -JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
 -JVM_OPTS="$JVM_OPTS -XX:CMSWaitDuration=10000"
 -
 -# note: bash evals '1.7.x' as > '1.7' so this is really a >= 1.7 jvm check
 -if { [ "$JVM_VERSION" \> "1.7" ] && [ "$JVM_VERSION" \< "1.8.0" ] && [ "$JVM_PATCH_VERSION" -ge "60" ]; } || [ "$JVM_VERSION" \> "1.8" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways -XX:CMSWaitDuration=10000"
 -fi
 -
 -if [ "$JVM_ARCH" = "64-Bit" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
 -fi
 -
 -# GC logging options
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDateStamps"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintHeapAtGC"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
 -#JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
 -JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
 -JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
 -# if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc-`date +%s`.log"
 -
  # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
  # JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StartupChecks.java
index e9f99ee,7ec16d1..03b362a
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@@ -17,9 -17,10 +17,11 @@@
   */
  package org.apache.cassandra.service;
  
 +import java.io.BufferedReader;
  import java.io.File;
  import java.io.IOException;
+ import java.lang.management.ManagementFactory;
+ import java.lang.management.RuntimeMXBean;
  import java.nio.file.*;
  import java.nio.file.attribute.BasicFileAttributes;
  import java.util.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/HeapUtils.java
index 67d7d79,2d068de..4c84f9b
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@@ -40,65 -35,44 +35,43 @@@ public final class HeapUtil
      private static final Logger logger = LoggerFactory.getLogger(HeapUtils.class);
  
      /**
--     * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
--     * or in the <code>CASSANDRA_HOME</code> directory.
++     * Generates a HEAP histogram in the log file.
       */
-     public static void generateHeapDump()
+     public static void logHeapHistogram()
      {
-         Long processId = getProcessId();
-         if (processId == null)
+         try
          {
-             logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-             return;
-         }
 -            logger.info("Trying to log the heap histogram using jmap");
++            logger.info("Trying to log the heap histogram using jcmd");
  
-         String heapDumpPath = getHeapDumpPathOption();
-         if (heapDumpPath == null)
-         {
-             String cassandraHome = System.getenv("CASSANDRA_HOME");
-             if (cassandraHome == null)
+             Long processId = getProcessId();
+             if (processId == null)
              {
+                 logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                  return;
              }
  
-             heapDumpPath = cassandraHome;
-         }
 -            String jmapPath = getJmapPath();
++            String jcmdPath = getJcmdPath();
  
-         Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-         if (Files.isDirectory(dumpPath))
-         {
-             dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-         }
 -            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
 -            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
++            // The jcmd file could not be found. In this case let's default to jcmd in the hope that it is in the path.
++            String jcmdCommand = jcmdPath == null ? "jcmd" : jcmdPath;
  
-         String jmapPath = getJmapPath();
 -            String[] histoCommands = new String[] {jmapCommand,
 -                    "-histo",
 -                    processId.toString()};
++            String[] histoCommands = new String[] {jcmdCommand,
++                    processId.toString(),
++                    "GC.class_histogram"};
  
-         // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-         String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
- 
-         String[] dumpCommands = new String[] {jmapCommand,
-                                               "-dump:format=b,file=" + dumpPath,
-                                               processId.toString()};
- 
-         // Lets also log the Heap histogram
-         String[] histoCommands = new String[] {jmapCommand,
-                                                "-histo",
-                                                processId.toString()};
-         try
-         {
-             logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
              logProcessOutput(Runtime.getRuntime().exec(histoCommands));
          }
-         catch (IOException e)
+         catch (Throwable e)
          {
-             logger.error("The heap dump could not be generated due to the following error: ", e);
+             logger.error("The heap histogram could not be generated due to the following error: ", e);
          }
      }
  
      /**
--     * Retrieve the path to the JMAP executable.
--     * @return the path to the JMAP executable or null if it cannot be found.
++     * Retrieve the path to the JCMD executable.
++     * @return the path to the JCMD executable or null if it cannot be found.
       */
--    private static String getJmapPath()
++    private static String getJcmdPath()
      {
          // Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
          // JVM might use the JRE which do not contains jmap.
@@@ -111,7 -85,7 +84,7 @@@
          {
              public boolean accept(File dir, String name)
              {
--                return name.startsWith("jmap");
++                return name.startsWith("jcmd");
              }
          });
          return ArrayUtils.isEmpty(files) ? null : files[0].getPath();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index c06a97b,0196b04..89ef129
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@@ -19,10 -19,9 +19,11 @@@ package org.apache.cassandra.utils
  
  import java.io.FileNotFoundException;
  import java.net.SocketException;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicBoolean;
  
  import com.google.common.annotations.VisibleForTesting;
+ 
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index eb8dbf9,f96ac6e..00447da
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@@ -27,8 -21,18 +27,10 @@@ import org.apache.cassandra.config.Conf
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.io.FSReadError;
  
+ import static java.util.Arrays.asList;
 -
 -import org.junit.Test;
 -
 -import java.io.FileNotFoundException;
 -import java.io.IOException;
 -import java.net.SocketException;
 -import java.util.Arrays;
 -
  import static org.junit.Assert.assertFalse;
  import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
  
  public class JVMStabilityInspectorTest
  {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[08/10] cassandra git commit: Merge branch cassandra-3.0 into cassandra-3.11

Posted by bl...@apache.org.
Merge branch cassandra-3.0 into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: a7c45be936bf8aa04b657529964c41b0d82c50e2
Parents: 88a41fb dd187d1
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:38:33 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:42:50 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 NEWS.txt                                        |   8 +-
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 914 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 632 insertions(+), 542 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 6e9a0bd,20ccc4b..116c7e9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,15 -1,4 +1,14 @@@
 -3.0.16
 +3.11.2
 + * Fix imbalanced disks when replacing node with same address with JBOD (CASSANDRA-14084)
 + * Reload compaction strategies when disk boundaries are invalidated (CASSANDRA-13948)
 + * Remove OpenJDK log warning (CASSANDRA-13916)
 + * Prevent compaction strategies from looping indefinitely (CASSANDRA-14079)
 + * Cache disk boundaries (CASSANDRA-13215)
 + * Add asm jar to build.xml for maven builds (CASSANDRA-11193)
 + * Round buffer size to powers of 2 for the chunk cache (CASSANDRA-13897)
 + * Update jackson JSON jars (CASSANDRA-13949)
 + * Avoid locks when checking LCS fanout and if we should defrag (CASSANDRA-13930)
 +Merged from 3.0:
- 3.0.16
   * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
   * Accept role names containing forward-slash (CASSANDRA-14088)
   * Optimize CRC check chance probability calculations (CASSANDRA-14094)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 9812b22,1bbe1aa..0c32278
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,7 -18,13 +18,13 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
-     - Nothing specific to this release, but please see previous upgrading sections.
 -   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 -     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 -     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 -     options are used. See CASSANDRA-13006 for more details.
 -   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 -     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 -     for more details.
++    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
++      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
++      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
++      options are used. See CASSANDRA-13006 for more details
++    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
++      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
++      for more details.
  
  Materialized Views
  -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/bin/cassandra
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index 2df7054,405ed92..49d03ce
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -393,70 -393,98 +393,80 @@@ Function SetCassandraEnvironmen
          $env:JVM_OPTS="$env:JVM_OPTS -XX:HeapDumpPath=$env:CASSANDRA_HEAPDUMP_DIR\cassandra-$unixTimestamp-pid$pid.hprof"
      }
  
-     if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
-     {
-         echo "Cassandra 3.0 and later require Java 8u40 or later."
-         exit
-     }
- 
-     # Specifies the default port over which Cassandra will be available for
-     # JMX connections.
-     $JMX_PORT="7199"
- 
-     # store in env to check if it's avail in verification
-     $env:JMX_PORT=$JMX_PORT
- 
-     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
-     # library path.
-     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
- 
-     # jmx: metrics and administration interface
-     #
-     # add this if you're having trouble connecting:
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
-     #
-     # see
-     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
-     # for more on configuring JMX through firewalls, etc. (Short version:
-     # get it working with no firewall first.)
-     #
-     # Due to potential security exploits, Cassandra ships with JMX accessible
-     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
-     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
-     #
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
-     #
-     # JMX SSL options
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
-     #
-     # JMX auth options
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-     ## Basic file based authn & authz
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.access.file=C:/jmxremote.access"
- 
-     ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
-     ## JAAS login modules can be used for authentication by uncommenting these two properties.
-     ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
-     ## which delegates to the IAuthenticator configured in cassandra.yaml
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djava.security.auth.login.config=C:/cassandra-jaas.config"
- 
-     ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
-     ## uncomment this to use it. Requires one of the two authentication options to be enabled
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
- 
-     # Default JMX setup, bound to local loopback address only
-     $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
- }
 -    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
 -    {
 -        echo "Cassandra 3.0 and later require Java 8u40 or later."
 -        exit
 -    }
 -
 -    # enable assertions.  disabling this in production will give a modest
 -    # performance benefit (around 5%).
 -    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
 -
 -    # Specifies the default port over which Cassandra will be available for
 -    # JMX connections.
 -    $JMX_PORT="7199"
 -
 -    # store in env to check if it's avail in verification
 -    $env:JMX_PORT=$JMX_PORT
 -
 -    # enable thread priorities, primarily so we can give periodic tasks
 -    # a lower priority to avoid interfering with client workload
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
 -    # allows lowering thread priority without being root on linux - probably
 -    # not necessary on Windows but doesn't harm anything.
 -    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 -
+     # stop the jvm on OutOfMemoryError as it can result in some data corruption
+     # uncomment the preferred option
+     # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+     # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+     # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+     $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+ 
+     # print an heap histogram on OutOfMemoryError
+     # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -    # Per-thread stack size.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 -
 -    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
 -
 -    # Make sure all memory is faulted and zeroed on startup.
 -    # This helps prevent soft faults in containers and makes
 -    # transparent hugepage allocation more effective.
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
 -
 -    # Biased locking does not benefit Cassandra.
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
++    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
++    {
++        echo "Cassandra 3.0 and later require Java 8u40 or later."
++        exit
++    }
+ 
 -    # Enable thread-local allocation blocks and allow the JVM to automatically
 -    # resize them at runtime.
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
++    # Specifies the default port over which Cassandra will be available for
++    # JMX connections.
++    $JMX_PORT="7199"
+ 
 -    # http://www.evanjones.ca/jvm-mmap-pause.html
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
++    # store in env to check if it's avail in verification
++    $env:JMX_PORT=$JMX_PORT
+ 
+     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
+     # library path.
+     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
+     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
+ 
 -    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -    # comment out this entry to enable IPv6 support).
 -    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
+     # jmx: metrics and administration interface
+     #
+     # add this if you're having trouble connecting:
+     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
+     #
+     # see
+     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
+     # for more on configuring JMX through firewalls, etc. (Short version:
+     # get it working with no firewall first.)
+     #
+     # Due to potential security exploits, Cassandra ships with JMX accessible
+     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
+     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
+     #
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
++    #
++    # JMX SSL options
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
++    #
++    # JMX auth options
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
++    ## Basic file based authn & authz
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.access.file=C:/jmxremote.access"
+ 
 -    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
++    ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
++    ## JAAS login modules can be used for authentication by uncommenting these two properties.
++    ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
++    ## which delegates to the IAuthenticator configured in cassandra.yaml
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djava.security.auth.login.config=C:/cassandra-jaas.config"
++
++    ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
++    ## uncomment this to use it. Requires one of the two authentication options to be enabled
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
+ 
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
++    # Default JMX setup, bound to local loopback address only
++    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
++
++    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --cc conf/cassandra-env.sh
index 5a02f79,ffc773f..189576b
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@@ -214,6 -247,30 +214,18 @@@ if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" 
      JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
  fi
  
+ # stop the jvm on OutOfMemoryError as it can result in some data corruption
+ # uncomment the preferred option
+ # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+ # on white spaces without taking quotes into account
+ # JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+ # JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+ JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+ 
+ # print an heap histogram on OutOfMemoryError
+ # JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -# uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -# JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -# uncomment to have Cassandra JVM log internal method compilation (developers only)
 -# JVM_OPTS="$JVM_OPTS -XX:+UnlockDiagnosticVMOptions -XX:+LogCompilation"
 -# JVM_OPTS="$JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 -
 -# Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -# http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -# comment out this entry to enable IPv6 support).
 -JVM_OPTS="$JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
  # jmx: metrics and administration interface
  #
  # add this if you're having trouble connecting:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[07/10] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
Merge branch cassandra-2.2 into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: dd187d105bce300395f52eb8ca4d8ccaf94a06ea
Parents: d7329a6 02aba73
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:30:29 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:32:36 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +-
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 950 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 650 insertions(+), 560 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 8cff716,5200eb1..20ccc4b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,83 -1,8 +1,84 @@@
 -2.2.12
 +3.0.16
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
+  * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
   * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  
 -2.2.11
 +
 +3.0.15
 + * Improve TRUNCATE performance (CASSANDRA-13909)
 + * Implement short read protection on partition boundaries (CASSANDRA-13595)
 + * Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries (CASSANDRA-13911)
 + * Filter header only commit logs before recovery (CASSANDRA-13918)
 + * AssertionError prepending to a list (CASSANDRA-13149)
 + * Fix support for SuperColumn tables (CASSANDRA-12373)
 + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883)
 + * Fix missing original update in TriggerExecutor (CASSANDRA-13894)
 + * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043)
 + * Improve short read protection performance (CASSANDRA-13794)
 + * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787)
 + * Fix short read protection for tables with no clustering columns (CASSANDRA-13880)
 + * Make isBuilt volatile in PartitionUpdate (CASSANDRA-13619)
 + * Prevent integer overflow of timestamps in CellTest and RowsTest (CASSANDRA-13866)
 + * Fix counter application order in short read protection (CASSANDRA-12872)
 + * Don't block RepairJob execution on validation futures (CASSANDRA-13797)
 + * Wait for all management tasks to complete before shutting down CLSM (CASSANDRA-13123)
 + * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
 + * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
 + * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
 + * Range deletes in a CAS batch are ignored (CASSANDRA-13655)
 + * Avoid assertion error when IndexSummary > 2G (CASSANDRA-12014)
 + * Change repair midpoint logging for tiny ranges (CASSANDRA-13603)
 + * Better handle corrupt final commitlog segment (CASSANDRA-11995)
 + * StreamingHistogram is not thread safe (CASSANDRA-13756)
 + * Fix MV timestamp issues (CASSANDRA-11500)
 + * Better tolerate improperly formatted bcrypt hashes (CASSANDRA-13626) 
 + * Fix race condition in read command serialization (CASSANDRA-13363)
 + * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
 + * Fix AssertionError in short read protection (CASSANDRA-13747)
 + * Don't skip corrupted sstables on startup (CASSANDRA-13620)
 + * Fix the merging of cells with different user type versions (CASSANDRA-13776)
 + * Copy session properties on cqlsh.py do_login (CASSANDRA-13640)
 + * Potential AssertionError during ReadRepair of range tombstone and partition deletions (CASSANDRA-13719)
 + * Don't let stress write warmup data if n=0 (CASSANDRA-13773)
 + * Gossip thread slows down when using batch commit log (CASSANDRA-12966)
 + * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
 + * Fix digest calculation for counter cells (CASSANDRA-13750)
 + * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
 + * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)
 + * Drop table should remove corresponding entries in dropped_columns table (CASSANDRA-13730)
 + * Log warn message until legacy auth tables have been migrated (CASSANDRA-13371)
 + * Fix incorrect [2.1 <- 3.0] serialization of counter cells created in 2.0 (CASSANDRA-13691)
 + * Fix invalid writetime for null cells (CASSANDRA-13711)
 + * Fix ALTER TABLE statement to atomically propagate changes to the table and its MVs (CASSANDRA-12952)
 + * Fixed ambiguous output of nodetool tablestats command (CASSANDRA-13722)
 + * JMXEnabledThreadPoolExecutor with corePoolSize equal to maxPoolSize (Backport CASSANDRA-13329)
 + * Fix Digest mismatch Exception if hints file has UnknownColumnFamily (CASSANDRA-13696)
 + * Purge tombstones created by expired cells (CASSANDRA-13643)
 + * Make concat work with iterators that have different subsets of columns (CASSANDRA-13482)
 + * Set test.runners based on cores and memory size (CASSANDRA-13078)
 + * Allow different NUMACTL_ARGS to be passed in (CASSANDRA-13557)
 + * Allow native function calls in CQLSSTableWriter (CASSANDRA-12606)
 + * Fix secondary index queries on COMPACT tables (CASSANDRA-13627)
 + * Nodetool listsnapshots output is missing a newline, if there are no snapshots (CASSANDRA-13568)
 + * sstabledump reports incorrect usage for argument order (CASSANDRA-13532)
 +Merged from 2.2:
   * Safely handle empty buffers when outputting to JSON (CASSANDRA-13868)
   * Copy session properties on cqlsh.py do_login (CASSANDRA-13847)
   * Fix load over calculated issue in IndexSummaryRedistribution (CASSANDRA-13738)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 621866b,5747941..1bbe1aa
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,66 -13,7 +13,71 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
-    - Nothing specific to this release, but please see previous upgrading sections,
-      especially if you are upgrading from 2.2.
++   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
++     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
++     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
++     options are used. See CASSANDRA-13006 for more details.
++   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
++     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
++     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/bin/cassandra
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index d3a0a87,7b4a632..405ed92
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -1,480 -1,482 +1,490 @@@
--#
--# 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.
--
--# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
--
--#-----------------------------------------------------------------------------
--Function SetCassandraHome()
--{
--    if (! $env:CASSANDRA_HOME)
--    {
--        $cwd = [System.IO.Directory]::GetCurrentDirectory()
--        $cwd = Split-Path $cwd -parent
--        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraMain()
--{
--    if (! $env:CASSANDRA_MAIN)
--    {
--        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function BuildClassPath
--{
--    $cp = """$env:CASSANDRA_HOME\conf"""
--    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
--    {
--        $file = $file -replace "\\", "/"
--        $cp = $cp + ";" + """$file"""
--    }
--
--    # Add build/classes/main so it works in development
--    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
--    $env:CLASSPATH=$cp
--}
--
--#-----------------------------------------------------------------------------
--Function CalculateHeapSizes
--{
--    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
--
--    $osInfo = Get-WmiObject -class "Win32_computersystem"
--    $autoPage = $osInfo.AutomaticManagedPageFile
--
--    if ($autoPage)
--    {
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--        echo ""
--        echo "    WARNING!  Automatic page file configuration detected."
--        echo "    It is recommended that you disable swap when running Cassandra"
--        echo "    for performance and stability reasons."
--        echo ""
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--    }
--    else
--    {
--        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
--        $pageFileCount = $PageFileInfo.Count
--        if ($pageFileInfo)
--        {
--            $files = @()
--            $sizes = @()
--            $hasSizes = $FALSE
--
--            # PageFileCount isn't populated and obj comes back as single if there's only 1
--            if ([string]::IsNullOrEmpty($PageFileCount))
--            {
--                $PageFileCount = 1
--                $files += $PageFileInfo.Name
--                if ($PageFileInfo.MaximumSize -ne 0)
--                {
--                    $hasSizes = $TRUE
--                    $sizes += $PageFileInfo.MaximumSize
--                }
--            }
--            else
--            {
--                for ($i = 0; $i -le $PageFileCount; $i++)
--                {
--                    $files += $PageFileInfo[$i].Name
--                    if ($PageFileInfo[$i].MaximumSize -ne 0)
--                    {
--                        $hasSizes = $TRUE
--                        $sizes += $PageFileInfo[$i].MaximumSize
--                    }
--                }
--            }
--
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING!  $PageFileCount swap file(s) detected"
--            for ($i = 0; $i -lt $PageFileCount; $i++)
--            {
--                $toPrint = "        Name: " + $files[$i]
--                if ($hasSizes)
--                {
--                    $toPrint = $toPrint + " Size: " + $sizes[$i]
--                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
--                }
--                echo $toPrint
--            }
--            echo "    It is recommended that you disable swap when running Cassandra"
--            echo "    for performance and stability reasons."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
--    }
--
--    # Validate that we need to run this function and that our config is good
--    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
--    {
--        return
--    }
- 
-     if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
 -    if (($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE))
--    {
--        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
--        exit 1
--    }
--
--    $memObject = Get-WMIObject -class win32_physicalmemory
--    if ($memObject -eq $null)
--    {
-         echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
 -        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\cassandra-env.ps1 for different heap values."
--        $env:MAX_HEAP_SIZE = "2048M"
--        $env:HEAP_NEWSIZE = "512M"
--        return
--    }
--
--    $memory = ($memObject | Measure-Object Capacity -Sum).sum
--    $memoryMB = [Math]::Truncate($memory / (1024*1024))
--
--    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
--    $systemCores = $cpu.NumberOfLogicalProcessors
--
--    # set max heap size based on the following
--    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
--    # calculate 1/2 ram and cap to 1024MB
--    # calculate 1/4 ram and cap to 8192MB
--    # pick the max
--    $halfMem = [Math]::Truncate($memoryMB / 2)
--    $quarterMem = [Math]::Truncate($halfMem / 2)
--
--    if ($halfMem -gt 1024)
--    {
--        $halfMem = 1024
--    }
--    if ($quarterMem -gt 8192)
--    {
--        $quarterMem = 8192
--    }
--
--    $maxHeapMB = ""
--    if ($halfMem -gt $quarterMem)
--    {
--        $maxHeapMB = $halfMem
--    }
--    else
--    {
--        $maxHeapMB = $quarterMem
--    }
--    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
--
--    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
--    $maxYGPerCore = 100
--    $maxYGTotal = $maxYGPerCore * $systemCores
--    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
--
--    if ($desiredYG -gt $maxYGTotal)
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
--    }
--    else
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
 -    }
 -}
 -
 -#-----------------------------------------------------------------------------
 -Function SetJsr223Env
 -{
 -    $cp = $env:CLASSPATH
 -    foreach ($jsrDir in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223")
 -    {
 -        foreach ($file in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223\$jsrDir\*.jar")
 -        {
 -            $file = $file -replace "\\", "/"
 -			$cp = $cp + ";" + """$file"""
 -        }
--    }
 -    $env:CLASSPATH=$cp
 -
 -	# JSR223/JRuby - set ruby lib directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\ruby")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djruby.lib=$env:CASSANDRA_HOME\lib\jsr223\jruby"
 -	}
 -	# JSR223/JRuby - set ruby JNI libraries root directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\jni")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djffi.boot.library.path=$env:CASSANDRA_HOME\lib\jsr223\jruby\jni"
 -	}
 -	# JSR223/Jython - set python.home system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jython\jython.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dpython.home=$env:CASSANDRA_HOME\lib\jsr223\jython"
 -	}
 -	# JSR223/Scala - necessary system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\scala\scala-compiler.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dscala.usejavacp=true"
 -	}
--}
--
--#-----------------------------------------------------------------------------
--Function ParseJVMInfo
--{
--    # grab info about the JVM
--    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
--    $pinfo.FileName = "$env:JAVA_BIN"
--    $pinfo.RedirectStandardError = $true
--    $pinfo.RedirectStandardOutput = $true
--    $pinfo.UseShellExecute = $false
--    $pinfo.Arguments = "-d64 -version"
--    $p = New-Object System.Diagnostics.Process
--    $p.StartInfo = $pinfo
--    $p.Start() | Out-Null
--    $p.WaitForExit()
--    $stderr = $p.StandardError.ReadToEnd()
--
--    $env:JVM_ARCH = "64-bit"
--
--    if ($stderr.Contains("Error"))
--    {
--        # 32-bit JVM. re-run w/out -d64
--        echo "Failed 64-bit check. Re-running to get version from 32-bit"
--        $pinfo.Arguments = "-version"
--        $p = New-Object System.Diagnostics.Process
--        $p.StartInfo = $pinfo
--        $p.Start() | Out-Null
--        $p.WaitForExit()
--        $stderr = $p.StandardError.ReadToEnd()
--        $env:JVM_ARCH = "32-bit"
--    }
--
--    $sa = $stderr.Split("""")
--    $env:JVM_VERSION = $sa[1]
--
--    if ($stderr.Contains("OpenJDK"))
--    {
--        $env:JVM_VENDOR = "OpenJDK"
--    }
--    elseif ($stderr.Contains("Java(TM)"))
--    {
--        $env:JVM_VENDOR = "Oracle"
--    }
--    else
--    {
--        $JVM_VENDOR = "other"
--    }
--
--    $pa = $sa[1].Split("_")
-     $subVersion = $pa[1]
-     # Deal with -b (build) versions
-     if ($subVersion -contains '-')
-     {
-         $patchAndBuild = $subVersion.Split("-")
-         $subVersion = $patchAndBuild[0]
-     }
-     $env:JVM_PATCH_VERSION = $subVersion
 -    $env:JVM_PATCH_VERSION=$pa[1]
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraEnvironment
--{
--    if (Test-Path Env:\JAVA_HOME)
--    {
--        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
--    }
--    elseif (Get-Command "java.exe")
--    {
--        $env:JAVA_BIN = "java.exe"
--    }
--    else
--    {
--        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
--        exit
--    }
--    SetCassandraHome
--    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
--    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
--
--    $logdir = "$env:CASSANDRA_HOME\logs"
--    $storagedir = "$env:CASSANDRA_HOME\data"
--    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
--
--    SetCassandraMain
--    BuildClassPath
 -    SetJsr223Env
--
--    # Override these to set the amount of memory to allocate to the JVM at
--    # start-up. For production use you may wish to adjust this for your
--    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
-     # to the Java heap. HEAP_NEWSIZE refers to the size of the young
 -    # to the Java heap; HEAP_NEWSIZE refers to the size of the young
--    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
--    # or not (if you set one, set the other).
--    #
--    # The main trade-off for the young generation is that the larger it
--    # is, the longer GC pause times will be. The shorter it is, the more
--    # expensive GC will be (usually).
--    #
--    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
--    # times. If in doubt, and if you do not particularly want to tweak, go
--    # 100 MB per physical CPU core.
- 
-     #GC log path has to be defined here since it needs to find CASSANDRA_HOME
-     $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
- 
-     # Read user-defined JVM options from jvm.options file
-     $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
-     for ($i = 0; $i -lt $content.Count; $i++)
-     {
-         $line = $content[$i]
-         if ($line.StartsWith("-"))
-         {
-             $env:JVM_OPTS = "$env:JVM_OPTS $line"
-         }
-     }
- 
-     $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
-     $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
-     $defined_xms = $env:JVM_OPTS -like '*Xms*'
-     $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
--
--    #$env:MAX_HEAP_SIZE="4096M"
--    #$env:HEAP_NEWSIZE="800M"
--    CalculateHeapSizes
--
--    ParseJVMInfo
- 
-     # We only set -Xms and -Xmx if they were not defined on jvm.options file
-     # If defined, both Xmx and Xms should be defined together.
-     if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
-     }
-     elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
-         exit
-     }
- 
-     # We only set -Xmn flag if it was not defined in jvm.options file
-     # and if the CMS GC is being used
-     # If defined, both Xmn and Xmx should be defined together.
-     if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
-         exit
-     }
-     elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
-     }
- 
-     if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
-     }
- 
--    # Add sigar env - see Cassandra-7838
--    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
--
--    # Confirm we're on high performance power plan, warn if not
--    # Change to $true to suppress this warning
--    $suppressPowerWarning = $false
--    if (!$suppressPowerWarning)
--    {
--        $currentProfile = powercfg /GETACTIVESCHEME
--        if (!$currentProfile.Contains("High performance"))
--        {
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING! Detected a power profile other than High Performance."
--            echo "    Performance of this node will suffer."
--            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
 -    }
 -
 -    # add the jamm javaagent
 -    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
 -        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
 -    {
 -        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
 -    }
 -
 -    # enable assertions.  disabling this in production will give a modest
 -    # performance benefit (around 5%).
 -    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
 -
 -    # Specifies the default port over which Cassandra will be available for
 -    # JMX connections.
 -    $JMX_PORT="7199"
 -
 -    # store in env to check if it's avail in verification
 -    $env:JMX_PORT=$JMX_PORT
 -
 -    # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled"
 -
 -    # enable thread priorities, primarily so we can give periodic tasks
 -    # a lower priority to avoid interfering with client workload
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
 -    # allows lowering thread priority without being root on linux - probably
 -    # not necessary on Windows but doesn't harm anything.
 -    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
 -
 -    # min and max heap sizes should be set to the same value to avoid
 -    # stop-the-world GC pauses during resize.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 -
 -    # stop the jvm on OutOfMemoryError as it can result in some data corruption
 -    # uncomment the preferred option
 -    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
 -
 -    # print an heap histogram on OutOfMemoryError
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
 -
 -    # Per-thread stack size.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 -
 -    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
 -
 -    # GC tuning options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseParNewGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseConcMarkSweepGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelRemarkEnabled"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:SurvivorRatio=8"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:MaxTenuringThreshold=1"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB"
 -    if (($env:JVM_VERSION.CompareTo("1.7") -eq 1) -and ($env:JVM_ARCH -eq "64-Bit"))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
 -    }
 -    if ( (($env:JVM_VERSION.CompareTo("1.7") -ge 0) -and ($env:JVM_PATCH_VERSION.CompareTo("60") -ge 0)) -or
 -         ($env:JVM_VERSION.CompareTo("1.8") -ge 0))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways"
--    }
 -
 -    # GC logging options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDetails"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDateStamps"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintHeapAtGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintTenuringDistribution"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintPromotionFailure"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseGCLogFileRotation"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:GCLogFileSize=10M"
 -    # if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -    # $currentDate = (Get-Date).ToString('yyyy.MM.dd')
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:$env:CASSANDRA_HOME/logs/gc-$currentDate.log"
 -
 -    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
 -    # library path.
 -    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
 -
 -    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -    # comment out this entry to enable IPv6 support).
 -    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
 -    # jmx: metrics and administration interface
 -    #
 -    # add this if you're having trouble connecting:
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
 -    #
 -    # see
 -    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
 -    # for more on configuring JMX through firewalls, etc. (Short version:
 -    # get it working with no firewall first.)
 -    #
 -    # Due to potential security exploits, Cassandra ships with JMX accessible
 -    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
 -    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity 
 -    #
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
--
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 -}
++#
++# 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.
++
++# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
++
++#-----------------------------------------------------------------------------
++Function SetCassandraHome()
++{
++    if (! $env:CASSANDRA_HOME)
++    {
++        $cwd = [System.IO.Directory]::GetCurrentDirectory()
++        $cwd = Split-Path $cwd -parent
++        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraMain()
++{
++    if (! $env:CASSANDRA_MAIN)
++    {
++        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function BuildClassPath
++{
++    $cp = """$env:CASSANDRA_HOME\conf"""
++    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
++    {
++        $file = $file -replace "\\", "/"
++        $cp = $cp + ";" + """$file"""
++    }
++
++    # Add build/classes/main so it works in development
++    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
++    $env:CLASSPATH=$cp
++}
++
++#-----------------------------------------------------------------------------
++Function CalculateHeapSizes
++{
++    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
++
++    $osInfo = Get-WmiObject -class "Win32_computersystem"
++    $autoPage = $osInfo.AutomaticManagedPageFile
++
++    if ($autoPage)
++    {
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++        echo ""
++        echo "    WARNING!  Automatic page file configuration detected."
++        echo "    It is recommended that you disable swap when running Cassandra"
++        echo "    for performance and stability reasons."
++        echo ""
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++    }
++    else
++    {
++        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
++        $pageFileCount = $PageFileInfo.Count
++        if ($pageFileInfo)
++        {
++            $files = @()
++            $sizes = @()
++            $hasSizes = $FALSE
++
++            # PageFileCount isn't populated and obj comes back as single if there's only 1
++            if ([string]::IsNullOrEmpty($PageFileCount))
++            {
++                $PageFileCount = 1
++                $files += $PageFileInfo.Name
++                if ($PageFileInfo.MaximumSize -ne 0)
++                {
++                    $hasSizes = $TRUE
++                    $sizes += $PageFileInfo.MaximumSize
++                }
++            }
++            else
++            {
++                for ($i = 0; $i -le $PageFileCount; $i++)
++                {
++                    $files += $PageFileInfo[$i].Name
++                    if ($PageFileInfo[$i].MaximumSize -ne 0)
++                    {
++                        $hasSizes = $TRUE
++                        $sizes += $PageFileInfo[$i].MaximumSize
++                    }
++                }
++            }
++
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING!  $PageFileCount swap file(s) detected"
++            for ($i = 0; $i -lt $PageFileCount; $i++)
++            {
++                $toPrint = "        Name: " + $files[$i]
++                if ($hasSizes)
++                {
++                    $toPrint = $toPrint + " Size: " + $sizes[$i]
++                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
++                }
++                echo $toPrint
++            }
++            echo "    It is recommended that you disable swap when running Cassandra"
++            echo "    for performance and stability reasons."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
++    # Validate that we need to run this function and that our config is good
++    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
++    {
++        return
++    }
++
++    if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
++    {
++        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
++        exit 1
++    }
++
++    $memObject = Get-WMIObject -class win32_physicalmemory
++    if ($memObject -eq $null)
++    {
++        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
++        $env:MAX_HEAP_SIZE = "2048M"
++        $env:HEAP_NEWSIZE = "512M"
++        return
++    }
++
++    $memory = ($memObject | Measure-Object Capacity -Sum).sum
++    $memoryMB = [Math]::Truncate($memory / (1024*1024))
++
++    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
++    $systemCores = $cpu.NumberOfLogicalProcessors
++
++    # set max heap size based on the following
++    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
++    # calculate 1/2 ram and cap to 1024MB
++    # calculate 1/4 ram and cap to 8192MB
++    # pick the max
++    $halfMem = [Math]::Truncate($memoryMB / 2)
++    $quarterMem = [Math]::Truncate($halfMem / 2)
++
++    if ($halfMem -gt 1024)
++    {
++        $halfMem = 1024
++    }
++    if ($quarterMem -gt 8192)
++    {
++        $quarterMem = 8192
++    }
++
++    $maxHeapMB = ""
++    if ($halfMem -gt $quarterMem)
++    {
++        $maxHeapMB = $halfMem
++    }
++    else
++    {
++        $maxHeapMB = $quarterMem
++    }
++    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
++
++    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
++    $maxYGPerCore = 100
++    $maxYGTotal = $maxYGPerCore * $systemCores
++    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
++
++    if ($desiredYG -gt $maxYGTotal)
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
++    }
++    else
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function ParseJVMInfo
++{
++    # grab info about the JVM
++    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
++    $pinfo.FileName = "$env:JAVA_BIN"
++    $pinfo.RedirectStandardError = $true
++    $pinfo.RedirectStandardOutput = $true
++    $pinfo.UseShellExecute = $false
++    $pinfo.Arguments = "-d64 -version"
++    $p = New-Object System.Diagnostics.Process
++    $p.StartInfo = $pinfo
++    $p.Start() | Out-Null
++    $p.WaitForExit()
++    $stderr = $p.StandardError.ReadToEnd()
++
++    $env:JVM_ARCH = "64-bit"
++
++    if ($stderr.Contains("Error"))
++    {
++        # 32-bit JVM. re-run w/out -d64
++        echo "Failed 64-bit check. Re-running to get version from 32-bit"
++        $pinfo.Arguments = "-version"
++        $p = New-Object System.Diagnostics.Process
++        $p.StartInfo = $pinfo
++        $p.Start() | Out-Null
++        $p.WaitForExit()
++        $stderr = $p.StandardError.ReadToEnd()
++        $env:JVM_ARCH = "32-bit"
++    }
++
++    $sa = $stderr.Split("""")
++    $env:JVM_VERSION = $sa[1]
++
++    if ($stderr.Contains("OpenJDK"))
++    {
++        $env:JVM_VENDOR = "OpenJDK"
++    }
++    elseif ($stderr.Contains("Java(TM)"))
++    {
++        $env:JVM_VENDOR = "Oracle"
++    }
++    else
++    {
++        $JVM_VENDOR = "other"
++    }
++
++    $pa = $sa[1].Split("_")
++    $subVersion = $pa[1]
++    # Deal with -b (build) versions
++    if ($subVersion -contains '-')
++    {
++        $patchAndBuild = $subVersion.Split("-")
++        $subVersion = $patchAndBuild[0]
++    }
++    $env:JVM_PATCH_VERSION = $subVersion
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraEnvironment
++{
++    if (Test-Path Env:\JAVA_HOME)
++    {
++        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
++    }
++    elseif (Get-Command "java.exe")
++    {
++        $env:JAVA_BIN = "java.exe"
++    }
++    else
++    {
++        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
++        exit
++    }
++    SetCassandraHome
++    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
++    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
++
++    $logdir = "$env:CASSANDRA_HOME\logs"
++    $storagedir = "$env:CASSANDRA_HOME\data"
++    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
++
++    SetCassandraMain
++    BuildClassPath
++
++    # Override these to set the amount of memory to allocate to the JVM at
++    # start-up. For production use you may wish to adjust this for your
++    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
++    # to the Java heap. HEAP_NEWSIZE refers to the size of the young
++    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
++    # or not (if you set one, set the other).
++    #
++    # The main trade-off for the young generation is that the larger it
++    # is, the longer GC pause times will be. The shorter it is, the more
++    # expensive GC will be (usually).
++    #
++    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
++    # times. If in doubt, and if you do not particularly want to tweak, go
++    # 100 MB per physical CPU core.
++
++    #GC log path has to be defined here since it needs to find CASSANDRA_HOME
++    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
++
++    # Read user-defined JVM options from jvm.options file
++    $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
++    for ($i = 0; $i -lt $content.Count; $i++)
++    {
++        $line = $content[$i]
++        if ($line.StartsWith("-"))
++        {
++            $env:JVM_OPTS = "$env:JVM_OPTS $line"
++        }
++    }
++
++    $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
++    $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
++    $defined_xms = $env:JVM_OPTS -like '*Xms*'
++    $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
++
++    #$env:MAX_HEAP_SIZE="4096M"
++    #$env:HEAP_NEWSIZE="800M"
++    CalculateHeapSizes
++
++    ParseJVMInfo
++
++    # We only set -Xms and -Xmx if they were not defined on jvm.options file
++    # If defined, both Xmx and Xms should be defined together.
++    if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
++    }
++    elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
++        exit
++    }
++
++    # We only set -Xmn flag if it was not defined in jvm.options file
++    # and if the CMS GC is being used
++    # If defined, both Xmn and Xmx should be defined together.
++    if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
++        exit
++    }
++    elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
++    }
++
++    if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
++    }
++
++    # Add sigar env - see Cassandra-7838
++    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
++
++    # Confirm we're on high performance power plan, warn if not
++    # Change to $true to suppress this warning
++    $suppressPowerWarning = $false
++    if (!$suppressPowerWarning)
++    {
++        $currentProfile = powercfg /GETACTIVESCHEME
++        if (!$currentProfile.Contains("High performance"))
++        {
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING! Detected a power profile other than High Performance."
++            echo "    Performance of this node will suffer."
++            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
 +    # provides hints to the JIT compiler
 +    $env:JVM_OPTS = "$env:JVM_OPTS -XX:CompileCommandFile=$env:CASSANDRA_CONF\hotspot_compiler"
 +
-     # add the jamm javaagent
-     if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
-         (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
-     {
-         $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
-     }
- 
++    # add the jamm javaagent
++    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
++        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
++    {
++        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
++    }
++
 +    # set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR
 +    if ($env:CASSANDRA_HEAPDUMP_DIR)
 +    {
 +        $unixTimestamp = [int64](([datetime]::UtcNow)-(get-date "1/1/1970")).TotalSeconds
 +        $env:JVM_OPTS="$env:JVM_OPTS -XX:HeapDumpPath=$env:CASSANDRA_HEAPDUMP_DIR\cassandra-$unixTimestamp-pid$pid.hprof"
 +    }
 +
-     if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
-     {
-         echo "Cassandra 3.0 and later require Java 8u40 or later."
-         exit
-     }
- 
-     # enable assertions.  disabling this in production will give a modest
-     # performance benefit (around 5%).
-     $env:JVM_OPTS = "$env:JVM_OPTS -ea"
- 
-     # Specifies the default port over which Cassandra will be available for
-     # JMX connections.
-     $JMX_PORT="7199"
- 
-     # store in env to check if it's avail in verification
-     $env:JMX_PORT=$JMX_PORT
- 
-     # enable thread priorities, primarily so we can give periodic tasks
-     # a lower priority to avoid interfering with client workload
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
-     # allows lowering thread priority without being root on linux - probably
-     # not necessary on Windows but doesn't harm anything.
-     # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
- 
-     # Per-thread stack size.
-     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
- 
-     # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
- 
-     # Make sure all memory is faulted and zeroed on startup.
-     # This helps prevent soft faults in containers and makes
-     # transparent hugepage allocation more effective.
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
- 
-     # Biased locking does not benefit Cassandra.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
- 
-     # Enable thread-local allocation blocks and allow the JVM to automatically
-     # resize them at runtime.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
- 
-     # http://www.evanjones.ca/jvm-mmap-pause.html
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
- 
-     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
-     # library path.
-     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
- 
-     # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
-     # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
- 
-     # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
-     # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
-     # comment out this entry to enable IPv6 support).
-     $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
- 
-     # jmx: metrics and administration interface
-     #
-     # add this if you're having trouble connecting:
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
-     #
-     # see
-     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
-     # for more on configuring JMX through firewalls, etc. (Short version:
-     # get it working with no firewall first.)
-     #
-     # Due to potential security exploits, Cassandra ships with JMX accessible
-     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
-     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
-     #
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-     $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
- 
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
- }
++    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
++    {
++        echo "Cassandra 3.0 and later require Java 8u40 or later."
++        exit
++    }
++
++    # enable assertions.  disabling this in production will give a modest
++    # performance benefit (around 5%).
++    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
++
++    # Specifies the default port over which Cassandra will be available for
++    # JMX connections.
++    $JMX_PORT="7199"
++
++    # store in env to check if it's avail in verification
++    $env:JMX_PORT=$JMX_PORT
++
++    # enable thread priorities, primarily so we can give periodic tasks
++    # a lower priority to avoid interfering with client workload
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
++    # allows lowering thread priority without being root on linux - probably
++    # not necessary on Windows but doesn't harm anything.
++    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
++
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
++
++    # stop the jvm on OutOfMemoryError as it can result in some data corruption
++    # uncomment the preferred option
++    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
++
++    # print an heap histogram on OutOfMemoryError
++    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
++
++    # Per-thread stack size.
++    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
++
++    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
++
++    # Make sure all memory is faulted and zeroed on startup.
++    # This helps prevent soft faults in containers and makes
++    # transparent hugepage allocation more effective.
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
++
++    # Biased locking does not benefit Cassandra.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
++
++    # Enable thread-local allocation blocks and allow the JVM to automatically
++    # resize them at runtime.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
++
++    # http://www.evanjones.ca/jvm-mmap-pause.html
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
++
++    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
++    # library path.
++    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
++
++    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
++    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
++
++    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
++    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
++    # comment out this entry to enable IPv6 support).
++    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
++
++    # jmx: metrics and administration interface
++    #
++    # add this if you're having trouble connecting:
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
++    #
++    # see
++    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
++    # for more on configuring JMX through firewalls, etc. (Short version:
++    # get it working with no firewall first.)
++    #
++    # Due to potential security exploits, Cassandra ships with JMX accessible
++    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
++    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
++    #
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
++    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
++
++    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
++
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --cc conf/cassandra-env.sh
index 44fe110,7b1b8d3..ffc773f
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@@ -247,6 -201,65 +247,18 @@@ if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" 
      JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
  fi
  
 -
 -startswith() { [ "${1#$2}" != "$1" ]; }
 -
+ # stop the jvm on OutOfMemoryError as it can result in some data corruption
+ # uncomment the preferred option
++# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+ # on white spaces without taking quotes into account
 -# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+ # JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+ JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+ 
+ # print an heap histogram on OutOfMemoryError
+ # JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -# Per-thread stack size.
 -JVM_OPTS="$JVM_OPTS -Xss256k"
 -
 -# Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -JVM_OPTS="$JVM_OPTS -XX:StringTableSize=1000003"
 -
 -# GC tuning options
 -JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+UseConcMarkSweepGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+CMSParallelRemarkEnabled" 
 -JVM_OPTS="$JVM_OPTS -XX:SurvivorRatio=8" 
 -JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=1"
 -JVM_OPTS="$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -JVM_OPTS="$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -JVM_OPTS="$JVM_OPTS -XX:+UseTLAB"
 -JVM_OPTS="$JVM_OPTS -XX:+PerfDisableSharedMem"
 -JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
 -JVM_OPTS="$JVM_OPTS -XX:CMSWaitDuration=10000"
 -
 -# note: bash evals '1.7.x' as > '1.7' so this is really a >= 1.7 jvm check
 -if { [ "$JVM_VERSION" \> "1.7" ] && [ "$JVM_VERSION" \< "1.8.0" ] && [ "$JVM_PATCH_VERSION" -ge "60" ]; } || [ "$JVM_VERSION" \> "1.8" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways -XX:CMSWaitDuration=10000"
 -fi
 -
 -if [ "$JVM_ARCH" = "64-Bit" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
 -fi
 -
 -# GC logging options
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDateStamps"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintHeapAtGC"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
 -#JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
 -JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
 -JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
 -# if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc-`date +%s`.log"
 -
  # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
  # JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StartupChecks.java
index e9f99ee,7ec16d1..03b362a
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@@ -17,9 -17,10 +17,11 @@@
   */
  package org.apache.cassandra.service;
  
 +import java.io.BufferedReader;
  import java.io.File;
  import java.io.IOException;
+ import java.lang.management.ManagementFactory;
+ import java.lang.management.RuntimeMXBean;
  import java.nio.file.*;
  import java.nio.file.attribute.BasicFileAttributes;
  import java.util.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/HeapUtils.java
index 67d7d79,2d068de..4c84f9b
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@@ -40,65 -35,44 +35,43 @@@ public final class HeapUtil
      private static final Logger logger = LoggerFactory.getLogger(HeapUtils.class);
  
      /**
--     * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
--     * or in the <code>CASSANDRA_HOME</code> directory.
++     * Generates a HEAP histogram in the log file.
       */
-     public static void generateHeapDump()
+     public static void logHeapHistogram()
      {
-         Long processId = getProcessId();
-         if (processId == null)
+         try
          {
-             logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-             return;
-         }
 -            logger.info("Trying to log the heap histogram using jmap");
++            logger.info("Trying to log the heap histogram using jcmd");
  
-         String heapDumpPath = getHeapDumpPathOption();
-         if (heapDumpPath == null)
-         {
-             String cassandraHome = System.getenv("CASSANDRA_HOME");
-             if (cassandraHome == null)
+             Long processId = getProcessId();
+             if (processId == null)
              {
+                 logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                  return;
              }
  
-             heapDumpPath = cassandraHome;
-         }
 -            String jmapPath = getJmapPath();
++            String jcmdPath = getJcmdPath();
  
-         Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-         if (Files.isDirectory(dumpPath))
-         {
-             dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-         }
 -            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
 -            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
++            // The jcmd file could not be found. In this case let's default to jcmd in the hope that it is in the path.
++            String jcmdCommand = jcmdPath == null ? "jcmd" : jcmdPath;
  
-         String jmapPath = getJmapPath();
 -            String[] histoCommands = new String[] {jmapCommand,
 -                    "-histo",
 -                    processId.toString()};
++            String[] histoCommands = new String[] {jcmdCommand,
++                    processId.toString(),
++                    "GC.class_histogram"};
  
-         // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-         String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
- 
-         String[] dumpCommands = new String[] {jmapCommand,
-                                               "-dump:format=b,file=" + dumpPath,
-                                               processId.toString()};
- 
-         // Lets also log the Heap histogram
-         String[] histoCommands = new String[] {jmapCommand,
-                                                "-histo",
-                                                processId.toString()};
-         try
-         {
-             logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
              logProcessOutput(Runtime.getRuntime().exec(histoCommands));
          }
-         catch (IOException e)
+         catch (Throwable e)
          {
-             logger.error("The heap dump could not be generated due to the following error: ", e);
+             logger.error("The heap histogram could not be generated due to the following error: ", e);
          }
      }
  
      /**
--     * Retrieve the path to the JMAP executable.
--     * @return the path to the JMAP executable or null if it cannot be found.
++     * Retrieve the path to the JCMD executable.
++     * @return the path to the JCMD executable or null if it cannot be found.
       */
--    private static String getJmapPath()
++    private static String getJcmdPath()
      {
          // Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
          // JVM might use the JRE which do not contains jmap.
@@@ -111,7 -85,7 +84,7 @@@
          {
              public boolean accept(File dir, String name)
              {
--                return name.startsWith("jmap");
++                return name.startsWith("jcmd");
              }
          });
          return ArrayUtils.isEmpty(files) ? null : files[0].getPath();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index c06a97b,0196b04..89ef129
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@@ -19,10 -19,9 +19,11 @@@ package org.apache.cassandra.utils
  
  import java.io.FileNotFoundException;
  import java.net.SocketException;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicBoolean;
  
  import com.google.common.annotations.VisibleForTesting;
+ 
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index eb8dbf9,f96ac6e..00447da
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@@ -27,8 -21,18 +27,10 @@@ import org.apache.cassandra.config.Conf
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.io.FSReadError;
  
+ import static java.util.Arrays.asList;
 -
 -import org.junit.Test;
 -
 -import java.io.FileNotFoundException;
 -import java.io.IOException;
 -import java.net.SocketException;
 -import java.util.Arrays;
 -
  import static org.junit.Assert.assertFalse;
  import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
  
  public class JVMStabilityInspectorTest
  {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[06/10] cassandra git commit: Merge branch cassandra-2.2 into cassandra-3.0

Posted by bl...@apache.org.
Merge branch cassandra-2.2 into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: dd187d105bce300395f52eb8ca4d8ccaf94a06ea
Parents: d7329a6 02aba73
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:30:29 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:32:36 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +-
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 950 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 650 insertions(+), 560 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 8cff716,5200eb1..20ccc4b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,83 -1,8 +1,84 @@@
 -2.2.12
 +3.0.16
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
+  * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
   * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  
 -2.2.11
 +
 +3.0.15
 + * Improve TRUNCATE performance (CASSANDRA-13909)
 + * Implement short read protection on partition boundaries (CASSANDRA-13595)
 + * Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries (CASSANDRA-13911)
 + * Filter header only commit logs before recovery (CASSANDRA-13918)
 + * AssertionError prepending to a list (CASSANDRA-13149)
 + * Fix support for SuperColumn tables (CASSANDRA-12373)
 + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883)
 + * Fix missing original update in TriggerExecutor (CASSANDRA-13894)
 + * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043)
 + * Improve short read protection performance (CASSANDRA-13794)
 + * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787)
 + * Fix short read protection for tables with no clustering columns (CASSANDRA-13880)
 + * Make isBuilt volatile in PartitionUpdate (CASSANDRA-13619)
 + * Prevent integer overflow of timestamps in CellTest and RowsTest (CASSANDRA-13866)
 + * Fix counter application order in short read protection (CASSANDRA-12872)
 + * Don't block RepairJob execution on validation futures (CASSANDRA-13797)
 + * Wait for all management tasks to complete before shutting down CLSM (CASSANDRA-13123)
 + * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
 + * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
 + * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
 + * Range deletes in a CAS batch are ignored (CASSANDRA-13655)
 + * Avoid assertion error when IndexSummary > 2G (CASSANDRA-12014)
 + * Change repair midpoint logging for tiny ranges (CASSANDRA-13603)
 + * Better handle corrupt final commitlog segment (CASSANDRA-11995)
 + * StreamingHistogram is not thread safe (CASSANDRA-13756)
 + * Fix MV timestamp issues (CASSANDRA-11500)
 + * Better tolerate improperly formatted bcrypt hashes (CASSANDRA-13626) 
 + * Fix race condition in read command serialization (CASSANDRA-13363)
 + * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
 + * Fix AssertionError in short read protection (CASSANDRA-13747)
 + * Don't skip corrupted sstables on startup (CASSANDRA-13620)
 + * Fix the merging of cells with different user type versions (CASSANDRA-13776)
 + * Copy session properties on cqlsh.py do_login (CASSANDRA-13640)
 + * Potential AssertionError during ReadRepair of range tombstone and partition deletions (CASSANDRA-13719)
 + * Don't let stress write warmup data if n=0 (CASSANDRA-13773)
 + * Gossip thread slows down when using batch commit log (CASSANDRA-12966)
 + * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
 + * Fix digest calculation for counter cells (CASSANDRA-13750)
 + * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
 + * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)
 + * Drop table should remove corresponding entries in dropped_columns table (CASSANDRA-13730)
 + * Log warn message until legacy auth tables have been migrated (CASSANDRA-13371)
 + * Fix incorrect [2.1 <- 3.0] serialization of counter cells created in 2.0 (CASSANDRA-13691)
 + * Fix invalid writetime for null cells (CASSANDRA-13711)
 + * Fix ALTER TABLE statement to atomically propagate changes to the table and its MVs (CASSANDRA-12952)
 + * Fixed ambiguous output of nodetool tablestats command (CASSANDRA-13722)
 + * JMXEnabledThreadPoolExecutor with corePoolSize equal to maxPoolSize (Backport CASSANDRA-13329)
 + * Fix Digest mismatch Exception if hints file has UnknownColumnFamily (CASSANDRA-13696)
 + * Purge tombstones created by expired cells (CASSANDRA-13643)
 + * Make concat work with iterators that have different subsets of columns (CASSANDRA-13482)
 + * Set test.runners based on cores and memory size (CASSANDRA-13078)
 + * Allow different NUMACTL_ARGS to be passed in (CASSANDRA-13557)
 + * Allow native function calls in CQLSSTableWriter (CASSANDRA-12606)
 + * Fix secondary index queries on COMPACT tables (CASSANDRA-13627)
 + * Nodetool listsnapshots output is missing a newline, if there are no snapshots (CASSANDRA-13568)
 + * sstabledump reports incorrect usage for argument order (CASSANDRA-13532)
 +Merged from 2.2:
   * Safely handle empty buffers when outputting to JSON (CASSANDRA-13868)
   * Copy session properties on cqlsh.py do_login (CASSANDRA-13847)
   * Fix load over calculated issue in IndexSummaryRedistribution (CASSANDRA-13738)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 621866b,5747941..1bbe1aa
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,66 -13,7 +13,71 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
-    - Nothing specific to this release, but please see previous upgrading sections,
-      especially if you are upgrading from 2.2.
++   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
++     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
++     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
++     options are used. See CASSANDRA-13006 for more details.
++   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
++     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
++     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/bin/cassandra
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index d3a0a87,7b4a632..405ed92
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -1,480 -1,482 +1,490 @@@
--#
--# 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.
--
--# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
--
--#-----------------------------------------------------------------------------
--Function SetCassandraHome()
--{
--    if (! $env:CASSANDRA_HOME)
--    {
--        $cwd = [System.IO.Directory]::GetCurrentDirectory()
--        $cwd = Split-Path $cwd -parent
--        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraMain()
--{
--    if (! $env:CASSANDRA_MAIN)
--    {
--        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
--    }
--}
--
--#-----------------------------------------------------------------------------
--Function BuildClassPath
--{
--    $cp = """$env:CASSANDRA_HOME\conf"""
--    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
--    {
--        $file = $file -replace "\\", "/"
--        $cp = $cp + ";" + """$file"""
--    }
--
--    # Add build/classes/main so it works in development
--    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
--    $env:CLASSPATH=$cp
--}
--
--#-----------------------------------------------------------------------------
--Function CalculateHeapSizes
--{
--    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
--
--    $osInfo = Get-WmiObject -class "Win32_computersystem"
--    $autoPage = $osInfo.AutomaticManagedPageFile
--
--    if ($autoPage)
--    {
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--        echo ""
--        echo "    WARNING!  Automatic page file configuration detected."
--        echo "    It is recommended that you disable swap when running Cassandra"
--        echo "    for performance and stability reasons."
--        echo ""
--        echo "*---------------------------------------------------------------------*"
--        echo "*---------------------------------------------------------------------*"
--    }
--    else
--    {
--        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
--        $pageFileCount = $PageFileInfo.Count
--        if ($pageFileInfo)
--        {
--            $files = @()
--            $sizes = @()
--            $hasSizes = $FALSE
--
--            # PageFileCount isn't populated and obj comes back as single if there's only 1
--            if ([string]::IsNullOrEmpty($PageFileCount))
--            {
--                $PageFileCount = 1
--                $files += $PageFileInfo.Name
--                if ($PageFileInfo.MaximumSize -ne 0)
--                {
--                    $hasSizes = $TRUE
--                    $sizes += $PageFileInfo.MaximumSize
--                }
--            }
--            else
--            {
--                for ($i = 0; $i -le $PageFileCount; $i++)
--                {
--                    $files += $PageFileInfo[$i].Name
--                    if ($PageFileInfo[$i].MaximumSize -ne 0)
--                    {
--                        $hasSizes = $TRUE
--                        $sizes += $PageFileInfo[$i].MaximumSize
--                    }
--                }
--            }
--
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING!  $PageFileCount swap file(s) detected"
--            for ($i = 0; $i -lt $PageFileCount; $i++)
--            {
--                $toPrint = "        Name: " + $files[$i]
--                if ($hasSizes)
--                {
--                    $toPrint = $toPrint + " Size: " + $sizes[$i]
--                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
--                }
--                echo $toPrint
--            }
--            echo "    It is recommended that you disable swap when running Cassandra"
--            echo "    for performance and stability reasons."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
--    }
--
--    # Validate that we need to run this function and that our config is good
--    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
--    {
--        return
--    }
- 
-     if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
 -    if (($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE))
--    {
--        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
--        exit 1
--    }
--
--    $memObject = Get-WMIObject -class win32_physicalmemory
--    if ($memObject -eq $null)
--    {
-         echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
 -        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\cassandra-env.ps1 for different heap values."
--        $env:MAX_HEAP_SIZE = "2048M"
--        $env:HEAP_NEWSIZE = "512M"
--        return
--    }
--
--    $memory = ($memObject | Measure-Object Capacity -Sum).sum
--    $memoryMB = [Math]::Truncate($memory / (1024*1024))
--
--    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
--    $systemCores = $cpu.NumberOfLogicalProcessors
--
--    # set max heap size based on the following
--    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
--    # calculate 1/2 ram and cap to 1024MB
--    # calculate 1/4 ram and cap to 8192MB
--    # pick the max
--    $halfMem = [Math]::Truncate($memoryMB / 2)
--    $quarterMem = [Math]::Truncate($halfMem / 2)
--
--    if ($halfMem -gt 1024)
--    {
--        $halfMem = 1024
--    }
--    if ($quarterMem -gt 8192)
--    {
--        $quarterMem = 8192
--    }
--
--    $maxHeapMB = ""
--    if ($halfMem -gt $quarterMem)
--    {
--        $maxHeapMB = $halfMem
--    }
--    else
--    {
--        $maxHeapMB = $quarterMem
--    }
--    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
--
--    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
--    $maxYGPerCore = 100
--    $maxYGTotal = $maxYGPerCore * $systemCores
--    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
--
--    if ($desiredYG -gt $maxYGTotal)
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
--    }
--    else
--    {
--        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
 -    }
 -}
 -
 -#-----------------------------------------------------------------------------
 -Function SetJsr223Env
 -{
 -    $cp = $env:CLASSPATH
 -    foreach ($jsrDir in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223")
 -    {
 -        foreach ($file in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223\$jsrDir\*.jar")
 -        {
 -            $file = $file -replace "\\", "/"
 -			$cp = $cp + ";" + """$file"""
 -        }
--    }
 -    $env:CLASSPATH=$cp
 -
 -	# JSR223/JRuby - set ruby lib directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\ruby")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djruby.lib=$env:CASSANDRA_HOME\lib\jsr223\jruby"
 -	}
 -	# JSR223/JRuby - set ruby JNI libraries root directory
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\jni")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djffi.boot.library.path=$env:CASSANDRA_HOME\lib\jsr223\jruby\jni"
 -	}
 -	# JSR223/Jython - set python.home system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jython\jython.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dpython.home=$env:CASSANDRA_HOME\lib\jsr223\jython"
 -	}
 -	# JSR223/Scala - necessary system property
 -	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\scala\scala-compiler.jar")
 -	{
 -		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dscala.usejavacp=true"
 -	}
--}
--
--#-----------------------------------------------------------------------------
--Function ParseJVMInfo
--{
--    # grab info about the JVM
--    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
--    $pinfo.FileName = "$env:JAVA_BIN"
--    $pinfo.RedirectStandardError = $true
--    $pinfo.RedirectStandardOutput = $true
--    $pinfo.UseShellExecute = $false
--    $pinfo.Arguments = "-d64 -version"
--    $p = New-Object System.Diagnostics.Process
--    $p.StartInfo = $pinfo
--    $p.Start() | Out-Null
--    $p.WaitForExit()
--    $stderr = $p.StandardError.ReadToEnd()
--
--    $env:JVM_ARCH = "64-bit"
--
--    if ($stderr.Contains("Error"))
--    {
--        # 32-bit JVM. re-run w/out -d64
--        echo "Failed 64-bit check. Re-running to get version from 32-bit"
--        $pinfo.Arguments = "-version"
--        $p = New-Object System.Diagnostics.Process
--        $p.StartInfo = $pinfo
--        $p.Start() | Out-Null
--        $p.WaitForExit()
--        $stderr = $p.StandardError.ReadToEnd()
--        $env:JVM_ARCH = "32-bit"
--    }
--
--    $sa = $stderr.Split("""")
--    $env:JVM_VERSION = $sa[1]
--
--    if ($stderr.Contains("OpenJDK"))
--    {
--        $env:JVM_VENDOR = "OpenJDK"
--    }
--    elseif ($stderr.Contains("Java(TM)"))
--    {
--        $env:JVM_VENDOR = "Oracle"
--    }
--    else
--    {
--        $JVM_VENDOR = "other"
--    }
--
--    $pa = $sa[1].Split("_")
-     $subVersion = $pa[1]
-     # Deal with -b (build) versions
-     if ($subVersion -contains '-')
-     {
-         $patchAndBuild = $subVersion.Split("-")
-         $subVersion = $patchAndBuild[0]
-     }
-     $env:JVM_PATCH_VERSION = $subVersion
 -    $env:JVM_PATCH_VERSION=$pa[1]
--}
--
--#-----------------------------------------------------------------------------
--Function SetCassandraEnvironment
--{
--    if (Test-Path Env:\JAVA_HOME)
--    {
--        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
--    }
--    elseif (Get-Command "java.exe")
--    {
--        $env:JAVA_BIN = "java.exe"
--    }
--    else
--    {
--        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
--        exit
--    }
--    SetCassandraHome
--    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
--    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
--
--    $logdir = "$env:CASSANDRA_HOME\logs"
--    $storagedir = "$env:CASSANDRA_HOME\data"
--    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
--
--    SetCassandraMain
--    BuildClassPath
 -    SetJsr223Env
--
--    # Override these to set the amount of memory to allocate to the JVM at
--    # start-up. For production use you may wish to adjust this for your
--    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
-     # to the Java heap. HEAP_NEWSIZE refers to the size of the young
 -    # to the Java heap; HEAP_NEWSIZE refers to the size of the young
--    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
--    # or not (if you set one, set the other).
--    #
--    # The main trade-off for the young generation is that the larger it
--    # is, the longer GC pause times will be. The shorter it is, the more
--    # expensive GC will be (usually).
--    #
--    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
--    # times. If in doubt, and if you do not particularly want to tweak, go
--    # 100 MB per physical CPU core.
- 
-     #GC log path has to be defined here since it needs to find CASSANDRA_HOME
-     $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
- 
-     # Read user-defined JVM options from jvm.options file
-     $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
-     for ($i = 0; $i -lt $content.Count; $i++)
-     {
-         $line = $content[$i]
-         if ($line.StartsWith("-"))
-         {
-             $env:JVM_OPTS = "$env:JVM_OPTS $line"
-         }
-     }
- 
-     $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
-     $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
-     $defined_xms = $env:JVM_OPTS -like '*Xms*'
-     $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
--
--    #$env:MAX_HEAP_SIZE="4096M"
--    #$env:HEAP_NEWSIZE="800M"
--    CalculateHeapSizes
--
--    ParseJVMInfo
- 
-     # We only set -Xms and -Xmx if they were not defined on jvm.options file
-     # If defined, both Xmx and Xms should be defined together.
-     if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
-     }
-     elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
-         exit
-     }
- 
-     # We only set -Xmn flag if it was not defined in jvm.options file
-     # and if the CMS GC is being used
-     # If defined, both Xmn and Xmx should be defined together.
-     if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
-     {
-         echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
-         exit
-     }
-     elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
-     }
- 
-     if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
-     {
-         $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
-     }
- 
--    # Add sigar env - see Cassandra-7838
--    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
--
--    # Confirm we're on high performance power plan, warn if not
--    # Change to $true to suppress this warning
--    $suppressPowerWarning = $false
--    if (!$suppressPowerWarning)
--    {
--        $currentProfile = powercfg /GETACTIVESCHEME
--        if (!$currentProfile.Contains("High performance"))
--        {
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--            echo ""
--            echo "    WARNING! Detected a power profile other than High Performance."
--            echo "    Performance of this node will suffer."
--            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
--            echo ""
--            echo "*---------------------------------------------------------------------*"
--            echo "*---------------------------------------------------------------------*"
--        }
 -    }
 -
 -    # add the jamm javaagent
 -    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
 -        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
 -    {
 -        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
 -    }
 -
 -    # enable assertions.  disabling this in production will give a modest
 -    # performance benefit (around 5%).
 -    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
 -
 -    # Specifies the default port over which Cassandra will be available for
 -    # JMX connections.
 -    $JMX_PORT="7199"
 -
 -    # store in env to check if it's avail in verification
 -    $env:JMX_PORT=$JMX_PORT
 -
 -    # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled"
 -
 -    # enable thread priorities, primarily so we can give periodic tasks
 -    # a lower priority to avoid interfering with client workload
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
 -    # allows lowering thread priority without being root on linux - probably
 -    # not necessary on Windows but doesn't harm anything.
 -    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
 -
 -    # min and max heap sizes should be set to the same value to avoid
 -    # stop-the-world GC pauses during resize.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 -
 -    # stop the jvm on OutOfMemoryError as it can result in some data corruption
 -    # uncomment the preferred option
 -    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
 -
 -    # print an heap histogram on OutOfMemoryError
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
 -
 -    # Per-thread stack size.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 -
 -    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
 -
 -    # GC tuning options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseParNewGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseConcMarkSweepGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelRemarkEnabled"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:SurvivorRatio=8"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:MaxTenuringThreshold=1"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB"
 -    if (($env:JVM_VERSION.CompareTo("1.7") -eq 1) -and ($env:JVM_ARCH -eq "64-Bit"))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
 -    }
 -    if ( (($env:JVM_VERSION.CompareTo("1.7") -ge 0) -and ($env:JVM_PATCH_VERSION.CompareTo("60") -ge 0)) -or
 -         ($env:JVM_VERSION.CompareTo("1.8") -ge 0))
 -    {
 -        $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways"
--    }
 -
 -    # GC logging options
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDetails"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDateStamps"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintHeapAtGC"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintTenuringDistribution"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintPromotionFailure"
 -    # $env:JVM_OPTS="$env:JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseGCLogFileRotation"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:GCLogFileSize=10M"
 -    # if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -    # $currentDate = (Get-Date).ToString('yyyy.MM.dd')
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:$env:CASSANDRA_HOME/logs/gc-$currentDate.log"
 -
 -    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
 -    # library path.
 -    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
 -
 -    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -    # comment out this entry to enable IPv6 support).
 -    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
 -    # jmx: metrics and administration interface
 -    #
 -    # add this if you're having trouble connecting:
 -    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
 -    #
 -    # see
 -    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
 -    # for more on configuring JMX through firewalls, etc. (Short version:
 -    # get it working with no firewall first.)
 -    #
 -    # Due to potential security exploits, Cassandra ships with JMX accessible
 -    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
 -    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity 
 -    #
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
--
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 -}
++#
++# 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.
++
++# NOTE: All param tuning can be done in the SetCassandraEnvironment Function below
++
++#-----------------------------------------------------------------------------
++Function SetCassandraHome()
++{
++    if (! $env:CASSANDRA_HOME)
++    {
++        $cwd = [System.IO.Directory]::GetCurrentDirectory()
++        $cwd = Split-Path $cwd -parent
++        $env:CASSANDRA_HOME = $cwd -replace "\\", "/"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraMain()
++{
++    if (! $env:CASSANDRA_MAIN)
++    {
++        $env:CASSANDRA_MAIN="org.apache.cassandra.service.CassandraDaemon"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function BuildClassPath
++{
++    $cp = """$env:CASSANDRA_HOME\conf"""
++    foreach ($file in Get-ChildItem "$env:CASSANDRA_HOME\lib\*.jar")
++    {
++        $file = $file -replace "\\", "/"
++        $cp = $cp + ";" + """$file"""
++    }
++
++    # Add build/classes/main so it works in development
++    $cp = $cp + ";" + """$env:CASSANDRA_HOME\build\classes\main"";""$env:CASSANDRA_HOME\build\classes\thrift"""
++    $env:CLASSPATH=$cp
++}
++
++#-----------------------------------------------------------------------------
++Function CalculateHeapSizes
++{
++    # Check if swapping is enabled on the host and warn if so - reference CASSANDRA-7316
++
++    $osInfo = Get-WmiObject -class "Win32_computersystem"
++    $autoPage = $osInfo.AutomaticManagedPageFile
++
++    if ($autoPage)
++    {
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++        echo ""
++        echo "    WARNING!  Automatic page file configuration detected."
++        echo "    It is recommended that you disable swap when running Cassandra"
++        echo "    for performance and stability reasons."
++        echo ""
++        echo "*---------------------------------------------------------------------*"
++        echo "*---------------------------------------------------------------------*"
++    }
++    else
++    {
++        $pageFileInfo = Get-WmiObject -class "Win32_PageFileSetting" -EnableAllPrivileges
++        $pageFileCount = $PageFileInfo.Count
++        if ($pageFileInfo)
++        {
++            $files = @()
++            $sizes = @()
++            $hasSizes = $FALSE
++
++            # PageFileCount isn't populated and obj comes back as single if there's only 1
++            if ([string]::IsNullOrEmpty($PageFileCount))
++            {
++                $PageFileCount = 1
++                $files += $PageFileInfo.Name
++                if ($PageFileInfo.MaximumSize -ne 0)
++                {
++                    $hasSizes = $TRUE
++                    $sizes += $PageFileInfo.MaximumSize
++                }
++            }
++            else
++            {
++                for ($i = 0; $i -le $PageFileCount; $i++)
++                {
++                    $files += $PageFileInfo[$i].Name
++                    if ($PageFileInfo[$i].MaximumSize -ne 0)
++                    {
++                        $hasSizes = $TRUE
++                        $sizes += $PageFileInfo[$i].MaximumSize
++                    }
++                }
++            }
++
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING!  $PageFileCount swap file(s) detected"
++            for ($i = 0; $i -lt $PageFileCount; $i++)
++            {
++                $toPrint = "        Name: " + $files[$i]
++                if ($hasSizes)
++                {
++                    $toPrint = $toPrint + " Size: " + $sizes[$i]
++                    $toPrint = $toPrint -replace [Environment]::NewLine, ""
++                }
++                echo $toPrint
++            }
++            echo "    It is recommended that you disable swap when running Cassandra"
++            echo "    for performance and stability reasons."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
++    # Validate that we need to run this function and that our config is good
++    if ($env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)
++    {
++        return
++    }
++
++    if ((($env:MAX_HEAP_SIZE -and !$env:HEAP_NEWSIZE) -or (!$env:MAX_HEAP_SIZE -and $env:HEAP_NEWSIZE)) -and ($using_cms -eq $true))
++    {
++        echo "Please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs.  Aborting startup."
++        exit 1
++    }
++
++    $memObject = Get-WMIObject -class win32_physicalmemory
++    if ($memObject -eq $null)
++    {
++        echo "WARNING!  Could not determine system memory.  Defaulting to 2G heap, 512M newgen.  Manually override in conf\jvm.options for different heap values."
++        $env:MAX_HEAP_SIZE = "2048M"
++        $env:HEAP_NEWSIZE = "512M"
++        return
++    }
++
++    $memory = ($memObject | Measure-Object Capacity -Sum).sum
++    $memoryMB = [Math]::Truncate($memory / (1024*1024))
++
++    $cpu = gwmi Win32_ComputerSystem | Select-Object NumberOfLogicalProcessors
++    $systemCores = $cpu.NumberOfLogicalProcessors
++
++    # set max heap size based on the following
++    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
++    # calculate 1/2 ram and cap to 1024MB
++    # calculate 1/4 ram and cap to 8192MB
++    # pick the max
++    $halfMem = [Math]::Truncate($memoryMB / 2)
++    $quarterMem = [Math]::Truncate($halfMem / 2)
++
++    if ($halfMem -gt 1024)
++    {
++        $halfMem = 1024
++    }
++    if ($quarterMem -gt 8192)
++    {
++        $quarterMem = 8192
++    }
++
++    $maxHeapMB = ""
++    if ($halfMem -gt $quarterMem)
++    {
++        $maxHeapMB = $halfMem
++    }
++    else
++    {
++        $maxHeapMB = $quarterMem
++    }
++    $env:MAX_HEAP_SIZE = [System.Convert]::ToString($maxHeapMB) + "M"
++
++    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4
++    $maxYGPerCore = 100
++    $maxYGTotal = $maxYGPerCore * $systemCores
++    $desiredYG = [Math]::Truncate($maxHeapMB / 4)
++
++    if ($desiredYG -gt $maxYGTotal)
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($maxYGTotal) + "M"
++    }
++    else
++    {
++        $env:HEAP_NEWSIZE = [System.Convert]::ToString($desiredYG) + "M"
++    }
++}
++
++#-----------------------------------------------------------------------------
++Function ParseJVMInfo
++{
++    # grab info about the JVM
++    $pinfo = New-Object System.Diagnostics.ProcessStartInfo
++    $pinfo.FileName = "$env:JAVA_BIN"
++    $pinfo.RedirectStandardError = $true
++    $pinfo.RedirectStandardOutput = $true
++    $pinfo.UseShellExecute = $false
++    $pinfo.Arguments = "-d64 -version"
++    $p = New-Object System.Diagnostics.Process
++    $p.StartInfo = $pinfo
++    $p.Start() | Out-Null
++    $p.WaitForExit()
++    $stderr = $p.StandardError.ReadToEnd()
++
++    $env:JVM_ARCH = "64-bit"
++
++    if ($stderr.Contains("Error"))
++    {
++        # 32-bit JVM. re-run w/out -d64
++        echo "Failed 64-bit check. Re-running to get version from 32-bit"
++        $pinfo.Arguments = "-version"
++        $p = New-Object System.Diagnostics.Process
++        $p.StartInfo = $pinfo
++        $p.Start() | Out-Null
++        $p.WaitForExit()
++        $stderr = $p.StandardError.ReadToEnd()
++        $env:JVM_ARCH = "32-bit"
++    }
++
++    $sa = $stderr.Split("""")
++    $env:JVM_VERSION = $sa[1]
++
++    if ($stderr.Contains("OpenJDK"))
++    {
++        $env:JVM_VENDOR = "OpenJDK"
++    }
++    elseif ($stderr.Contains("Java(TM)"))
++    {
++        $env:JVM_VENDOR = "Oracle"
++    }
++    else
++    {
++        $JVM_VENDOR = "other"
++    }
++
++    $pa = $sa[1].Split("_")
++    $subVersion = $pa[1]
++    # Deal with -b (build) versions
++    if ($subVersion -contains '-')
++    {
++        $patchAndBuild = $subVersion.Split("-")
++        $subVersion = $patchAndBuild[0]
++    }
++    $env:JVM_PATCH_VERSION = $subVersion
++}
++
++#-----------------------------------------------------------------------------
++Function SetCassandraEnvironment
++{
++    if (Test-Path Env:\JAVA_HOME)
++    {
++        $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"
++    }
++    elseif (Get-Command "java.exe")
++    {
++        $env:JAVA_BIN = "java.exe"
++    }
++    else
++    {
++        echo "ERROR!  No JAVA_HOME set and could not find java.exe in the path."
++        exit
++    }
++    SetCassandraHome
++    $env:CASSANDRA_CONF = "$env:CASSANDRA_HOME\conf"
++    $env:CASSANDRA_PARAMS="-Dcassandra -Dlogback.configurationFile=logback.xml"
++
++    $logdir = "$env:CASSANDRA_HOME\logs"
++    $storagedir = "$env:CASSANDRA_HOME\data"
++    $env:CASSANDRA_PARAMS = $env:CASSANDRA_PARAMS + " -Dcassandra.logdir=""$logdir"" -Dcassandra.storagedir=""$storagedir"""
++
++    SetCassandraMain
++    BuildClassPath
++
++    # Override these to set the amount of memory to allocate to the JVM at
++    # start-up. For production use you may wish to adjust this for your
++    # environment. MAX_HEAP_SIZE is the total amount of memory dedicated
++    # to the Java heap. HEAP_NEWSIZE refers to the size of the young
++    # generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
++    # or not (if you set one, set the other).
++    #
++    # The main trade-off for the young generation is that the larger it
++    # is, the longer GC pause times will be. The shorter it is, the more
++    # expensive GC will be (usually).
++    #
++    # The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent
++    # times. If in doubt, and if you do not particularly want to tweak, go
++    # 100 MB per physical CPU core.
++
++    #GC log path has to be defined here since it needs to find CASSANDRA_HOME
++    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""
++
++    # Read user-defined JVM options from jvm.options file
++    $content = Get-Content "$env:CASSANDRA_CONF\jvm.options"
++    for ($i = 0; $i -lt $content.Count; $i++)
++    {
++        $line = $content[$i]
++        if ($line.StartsWith("-"))
++        {
++            $env:JVM_OPTS = "$env:JVM_OPTS $line"
++        }
++    }
++
++    $defined_xmn = $env:JVM_OPTS -like '*Xmn*'
++    $defined_xmx = $env:JVM_OPTS -like '*Xmx*'
++    $defined_xms = $env:JVM_OPTS -like '*Xms*'
++    $using_cms = $env:JVM_OPTS -like '*UseConcMarkSweepGC*'
++
++    #$env:MAX_HEAP_SIZE="4096M"
++    #$env:HEAP_NEWSIZE="800M"
++    CalculateHeapSizes
++
++    ParseJVMInfo
++
++    # We only set -Xms and -Xmx if they were not defined on jvm.options file
++    # If defined, both Xmx and Xms should be defined together.
++    if (($defined_xmx -eq $false) -and ($defined_xms -eq $false))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xms$env:MAX_HEAP_SIZE"
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmx$env:MAX_HEAP_SIZE"
++    }
++    elseif (($defined_xmx -eq $false) -or ($defined_xms -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
++        exit
++    }
++
++    # We only set -Xmn flag if it was not defined in jvm.options file
++    # and if the CMS GC is being used
++    # If defined, both Xmn and Xmx should be defined together.
++    if (($defined_xmn -eq $true) -and ($defined_xmx -eq $false))
++    {
++        echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
++        exit
++    }
++    elseif (($defined_xmn -eq $false) -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
++    }
++
++    if (($env:JVM_ARCH -eq "64-Bit") -and ($using_cms -eq $true))
++    {
++        $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCondCardMark"
++    }
++
++    # Add sigar env - see Cassandra-7838
++    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""
++
++    # Confirm we're on high performance power plan, warn if not
++    # Change to $true to suppress this warning
++    $suppressPowerWarning = $false
++    if (!$suppressPowerWarning)
++    {
++        $currentProfile = powercfg /GETACTIVESCHEME
++        if (!$currentProfile.Contains("High performance"))
++        {
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++            echo ""
++            echo "    WARNING! Detected a power profile other than High Performance."
++            echo "    Performance of this node will suffer."
++            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."
++            echo ""
++            echo "*---------------------------------------------------------------------*"
++            echo "*---------------------------------------------------------------------*"
++        }
++    }
++
 +    # provides hints to the JIT compiler
 +    $env:JVM_OPTS = "$env:JVM_OPTS -XX:CompileCommandFile=$env:CASSANDRA_CONF\hotspot_compiler"
 +
-     # add the jamm javaagent
-     if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
-         (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
-     {
-         $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
-     }
- 
++    # add the jamm javaagent
++    if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or
++        (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))
++    {
++        $env:JVM_OPTS = "$env:JVM_OPTS -javaagent:""$env:CASSANDRA_HOME\lib\jamm-0.3.0.jar"""
++    }
++
 +    # set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR
 +    if ($env:CASSANDRA_HEAPDUMP_DIR)
 +    {
 +        $unixTimestamp = [int64](([datetime]::UtcNow)-(get-date "1/1/1970")).TotalSeconds
 +        $env:JVM_OPTS="$env:JVM_OPTS -XX:HeapDumpPath=$env:CASSANDRA_HEAPDUMP_DIR\cassandra-$unixTimestamp-pid$pid.hprof"
 +    }
 +
-     if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
-     {
-         echo "Cassandra 3.0 and later require Java 8u40 or later."
-         exit
-     }
- 
-     # enable assertions.  disabling this in production will give a modest
-     # performance benefit (around 5%).
-     $env:JVM_OPTS = "$env:JVM_OPTS -ea"
- 
-     # Specifies the default port over which Cassandra will be available for
-     # JMX connections.
-     $JMX_PORT="7199"
- 
-     # store in env to check if it's avail in verification
-     $env:JMX_PORT=$JMX_PORT
- 
-     # enable thread priorities, primarily so we can give periodic tasks
-     # a lower priority to avoid interfering with client workload
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
-     # allows lowering thread priority without being root on linux - probably
-     # not necessary on Windows but doesn't harm anything.
-     # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
- 
-     # Per-thread stack size.
-     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
- 
-     # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
- 
-     # Make sure all memory is faulted and zeroed on startup.
-     # This helps prevent soft faults in containers and makes
-     # transparent hugepage allocation more effective.
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
- 
-     # Biased locking does not benefit Cassandra.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
- 
-     # Enable thread-local allocation blocks and allow the JVM to automatically
-     # resize them at runtime.
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
- 
-     # http://www.evanjones.ca/jvm-mmap-pause.html
-     $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
- 
-     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
-     # library path.
-     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
- 
-     # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
-     # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
- 
-     # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
-     # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
-     # comment out this entry to enable IPv6 support).
-     $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
- 
-     # jmx: metrics and administration interface
-     #
-     # add this if you're having trouble connecting:
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
-     #
-     # see
-     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
-     # for more on configuring JMX through firewalls, etc. (Short version:
-     # get it working with no firewall first.)
-     #
-     # Due to potential security exploits, Cassandra ships with JMX accessible
-     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
-     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
-     #
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-     $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
- 
-     #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
- }
++    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
++    {
++        echo "Cassandra 3.0 and later require Java 8u40 or later."
++        exit
++    }
++
++    # enable assertions.  disabling this in production will give a modest
++    # performance benefit (around 5%).
++    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
++
++    # Specifies the default port over which Cassandra will be available for
++    # JMX connections.
++    $JMX_PORT="7199"
++
++    # store in env to check if it's avail in verification
++    $env:JMX_PORT=$JMX_PORT
++
++    # enable thread priorities, primarily so we can give periodic tasks
++    # a lower priority to avoid interfering with client workload
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
++    # allows lowering thread priority without being root on linux - probably
++    # not necessary on Windows but doesn't harm anything.
++    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
++
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
++
++    # stop the jvm on OutOfMemoryError as it can result in some data corruption
++    # uncomment the preferred option
++    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
++    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
++
++    # print an heap histogram on OutOfMemoryError
++    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
++
++    # Per-thread stack size.
++    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
++
++    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
++
++    # Make sure all memory is faulted and zeroed on startup.
++    # This helps prevent soft faults in containers and makes
++    # transparent hugepage allocation more effective.
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
++
++    # Biased locking does not benefit Cassandra.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
++
++    # Enable thread-local allocation blocks and allow the JVM to automatically
++    # resize them at runtime.
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
++
++    # http://www.evanjones.ca/jvm-mmap-pause.html
++    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
++
++    # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
++    # library path.
++    # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
++
++    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
++    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
++
++    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
++    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
++    # comment out this entry to enable IPv6 support).
++    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
++
++    # jmx: metrics and administration interface
++    #
++    # add this if you're having trouble connecting:
++    # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
++    #
++    # see
++    # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
++    # for more on configuring JMX through firewalls, etc. (Short version:
++    # get it working with no firewall first.)
++    #
++    # Due to potential security exploits, Cassandra ships with JMX accessible
++    # *only* from localhost.  To enable remote JMX connections, uncomment lines below
++    # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
++    #
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
++    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
++
++    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
++
++    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --cc conf/cassandra-env.sh
index 44fe110,7b1b8d3..ffc773f
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@@ -247,6 -201,65 +247,18 @@@ if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" 
      JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
  fi
  
 -
 -startswith() { [ "${1#$2}" != "$1" ]; }
 -
+ # stop the jvm on OutOfMemoryError as it can result in some data corruption
+ # uncomment the preferred option
++# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+ # on white spaces without taking quotes into account
 -# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+ # JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+ JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+ 
+ # print an heap histogram on OutOfMemoryError
+ # JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -# Per-thread stack size.
 -JVM_OPTS="$JVM_OPTS -Xss256k"
 -
 -# Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -JVM_OPTS="$JVM_OPTS -XX:StringTableSize=1000003"
 -
 -# GC tuning options
 -JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+UseConcMarkSweepGC" 
 -JVM_OPTS="$JVM_OPTS -XX:+CMSParallelRemarkEnabled" 
 -JVM_OPTS="$JVM_OPTS -XX:SurvivorRatio=8" 
 -JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=1"
 -JVM_OPTS="$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 -JVM_OPTS="$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 -JVM_OPTS="$JVM_OPTS -XX:+UseTLAB"
 -JVM_OPTS="$JVM_OPTS -XX:+PerfDisableSharedMem"
 -JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
 -JVM_OPTS="$JVM_OPTS -XX:CMSWaitDuration=10000"
 -
 -# note: bash evals '1.7.x' as > '1.7' so this is really a >= 1.7 jvm check
 -if { [ "$JVM_VERSION" \> "1.7" ] && [ "$JVM_VERSION" \< "1.8.0" ] && [ "$JVM_PATCH_VERSION" -ge "60" ]; } || [ "$JVM_VERSION" \> "1.8" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways -XX:CMSWaitDuration=10000"
 -fi
 -
 -if [ "$JVM_ARCH" = "64-Bit" ] ; then
 -    JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
 -fi
 -
 -# GC logging options
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCDateStamps"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintHeapAtGC"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
 -JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
 -#JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
 -
 -JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
 -JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
 -JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
 -JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
 -# if using version before JDK 6u34 or 7u2 use this instead of log rotation
 -# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc-`date +%s`.log"
 -
  # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
  # JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StartupChecks.java
index e9f99ee,7ec16d1..03b362a
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@@ -17,9 -17,10 +17,11 @@@
   */
  package org.apache.cassandra.service;
  
 +import java.io.BufferedReader;
  import java.io.File;
  import java.io.IOException;
+ import java.lang.management.ManagementFactory;
+ import java.lang.management.RuntimeMXBean;
  import java.nio.file.*;
  import java.nio.file.attribute.BasicFileAttributes;
  import java.util.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/HeapUtils.java
index 67d7d79,2d068de..4c84f9b
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@@ -40,65 -35,44 +35,43 @@@ public final class HeapUtil
      private static final Logger logger = LoggerFactory.getLogger(HeapUtils.class);
  
      /**
--     * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
--     * or in the <code>CASSANDRA_HOME</code> directory.
++     * Generates a HEAP histogram in the log file.
       */
-     public static void generateHeapDump()
+     public static void logHeapHistogram()
      {
-         Long processId = getProcessId();
-         if (processId == null)
+         try
          {
-             logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-             return;
-         }
 -            logger.info("Trying to log the heap histogram using jmap");
++            logger.info("Trying to log the heap histogram using jcmd");
  
-         String heapDumpPath = getHeapDumpPathOption();
-         if (heapDumpPath == null)
-         {
-             String cassandraHome = System.getenv("CASSANDRA_HOME");
-             if (cassandraHome == null)
+             Long processId = getProcessId();
+             if (processId == null)
              {
+                 logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                  return;
              }
  
-             heapDumpPath = cassandraHome;
-         }
 -            String jmapPath = getJmapPath();
++            String jcmdPath = getJcmdPath();
  
-         Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-         if (Files.isDirectory(dumpPath))
-         {
-             dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-         }
 -            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
 -            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
++            // The jcmd file could not be found. In this case let's default to jcmd in the hope that it is in the path.
++            String jcmdCommand = jcmdPath == null ? "jcmd" : jcmdPath;
  
-         String jmapPath = getJmapPath();
 -            String[] histoCommands = new String[] {jmapCommand,
 -                    "-histo",
 -                    processId.toString()};
++            String[] histoCommands = new String[] {jcmdCommand,
++                    processId.toString(),
++                    "GC.class_histogram"};
  
-         // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-         String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
- 
-         String[] dumpCommands = new String[] {jmapCommand,
-                                               "-dump:format=b,file=" + dumpPath,
-                                               processId.toString()};
- 
-         // Lets also log the Heap histogram
-         String[] histoCommands = new String[] {jmapCommand,
-                                                "-histo",
-                                                processId.toString()};
-         try
-         {
-             logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
              logProcessOutput(Runtime.getRuntime().exec(histoCommands));
          }
-         catch (IOException e)
+         catch (Throwable e)
          {
-             logger.error("The heap dump could not be generated due to the following error: ", e);
+             logger.error("The heap histogram could not be generated due to the following error: ", e);
          }
      }
  
      /**
--     * Retrieve the path to the JMAP executable.
--     * @return the path to the JMAP executable or null if it cannot be found.
++     * Retrieve the path to the JCMD executable.
++     * @return the path to the JCMD executable or null if it cannot be found.
       */
--    private static String getJmapPath()
++    private static String getJcmdPath()
      {
          // Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
          // JVM might use the JRE which do not contains jmap.
@@@ -111,7 -85,7 +84,7 @@@
          {
              public boolean accept(File dir, String name)
              {
--                return name.startsWith("jmap");
++                return name.startsWith("jcmd");
              }
          });
          return ArrayUtils.isEmpty(files) ? null : files[0].getPath();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index c06a97b,0196b04..89ef129
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@@ -19,10 -19,9 +19,11 @@@ package org.apache.cassandra.utils
  
  import java.io.FileNotFoundException;
  import java.net.SocketException;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicBoolean;
  
  import com.google.common.annotations.VisibleForTesting;
+ 
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd187d10/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index eb8dbf9,f96ac6e..00447da
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@@ -27,8 -21,18 +27,10 @@@ import org.apache.cassandra.config.Conf
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.io.FSReadError;
  
+ import static java.util.Arrays.asList;
 -
 -import org.junit.Test;
 -
 -import java.io.FileNotFoundException;
 -import java.io.IOException;
 -import java.net.SocketException;
 -import java.util.Arrays;
 -
  import static org.junit.Assert.assertFalse;
  import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
  
  public class JVMStabilityInspectorTest
  {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[03/10] cassandra git commit: Rely on the JVM to handle OutOfMemoryErrors

Posted by bl...@apache.org.
Rely on the JVM to handle OutOfMemoryErrors

patch by Benjamin Lerer; reviewed by Joshua McKenzie for CASSANDRA-13006


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

Branch: refs/heads/cassandra-3.11
Commit: 02aba7343ce300397ab672bbb1788aa8182d8a48
Parents: 3cd2c3c
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:21:05 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:21:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  9 ++-
 bin/cassandra                                   | 19 ++++-
 conf/cassandra-env.ps1                          | 10 +++
 conf/cassandra-env.sh                           | 12 +++
 .../apache/cassandra/service/StartupChecks.java | 74 ++++++++++++++++++
 .../org/apache/cassandra/utils/HeapUtils.java   | 82 ++++----------------
 .../cassandra/utils/JVMStabilityInspector.java  | 24 +++++-
 .../utils/JVMStabilityInspectorTest.java        | 31 +++++---
 9 files changed, 178 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1e81fd..5200eb1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.12
+ * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
  * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 
 2.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3bff458..5747941 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,8 +18,13 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.2 if you are upgrading
-      from a previous version.
+    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
+      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
+      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
+      options are used. See CASSANDRA-13006 for more details.
+    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
+      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
+      for more details.
 
 2.2.11
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/bin/cassandra
----------------------------------------------------------------------
diff --git a/bin/cassandra b/bin/cassandra
index 2dd0fe1..0e337e8 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -28,6 +28,7 @@
 #
 #   CLASSPATH -- A Java classpath containing everything necessary to run.
 #   JVM_OPTS -- Additional arguments to the JVM for heap size, etc
+#   JVM_ON_OUT_OF_MEMORY_ERROR_OPT -- The OnOutOfMemoryError JVM option if specified
 #   CASSANDRA_CONF -- Directory containing Cassandra configuration files.
 #
 # As a convenience, a fragment of shell is sourced in order to set one or
@@ -199,12 +200,22 @@ launch_service()
     # to close stdout/stderr, but it's up to us not to background.
     if [ "x$foreground" != "x" ]; then
         cassandra_parms="$cassandra_parms -Dcassandra-foreground=yes"
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        fi
     # Startup CassandraDaemon, background it, and write the pid.
     else
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
-        [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
-        true
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        fi
     fi
 
     return $?

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 321a9ca..7b4a632 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -390,6 +390,16 @@ Function SetCassandraEnvironment
     $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 
+    # stop the jvm on OutOfMemoryError as it can result in some data corruption
+    # uncomment the preferred option
+    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+
+    # print an heap histogram on OutOfMemoryError
+    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
     # Per-thread stack size.
     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index b519b76..7b1b8d3 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -204,6 +204,18 @@ fi
 
 startswith() { [ "${1#$2}" != "$1" ]; }
 
+# stop the jvm on OutOfMemoryError as it can result in some data corruption
+# uncomment the preferred option
+# For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+# on white spaces without taking quotes into account
+# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+# JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+# JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+
+# print an heap histogram on OutOfMemoryError
+# JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
 # Per-thread stack size.
 JVM_OPTS="$JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 34bc824..7ec16d1 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.service;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
 import java.nio.file.*;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
@@ -178,6 +180,78 @@ public class StartupChecks
             {
                 logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
             }
+            else
+            {
+                    checkOutOfMemoryHandling();
+            }
+        }
+
+        /**
+         * Checks that the JVM is configured to handle OutOfMemoryError
+         */
+        private void checkOutOfMemoryHandling()
+        {
+            int version = getJavaVersion();
+            int update = getUpdate();
+            // The ExitOnOutOfMemory and CrashOnOutOfMemory are supported since the version 7u101 and 8u92
+            boolean jreSupportExitOnOutOfMemory = version > 8
+                                                    || (version == 7 && update >= 101)
+                                                    || (version == 8 && update >= 92);
+            if (jreSupportExitOnOutOfMemory)
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError=", "-XX:+ExitOnOutOfMemoryError", "-XX:+CrashOnOutOfMemoryError"))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                                + " Use one of the following JVM options to configure the behavior on OutOfMemoryError: "
+                                + " -XX:+ExitOnOutOfMemoryError, -XX:+CrashOnOutOfMemoryError, or -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\"");
+            }
+            else
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError="))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                            + " Either upgrade your JRE to a version greater or equal to 8u92 and use -XX:+ExitOnOutOfMemoryError/-XX:+CrashOnOutOfMemoryError"
+                            + " or use -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\" on your current JRE.");
+            }
+        }
+
+        /**
+         * Returns the java version number for an Oracle JVM.
+         * @return the java version number
+         */
+        private int getJavaVersion()
+        {
+            String jreVersion = System.getProperty("java.version");
+            String version = jreVersion.startsWith("1.") ? jreVersion.substring(2, 3) // Pre 9 version
+                                                         : jreVersion.substring(0, jreVersion.indexOf('.'));
+            return Integer.parseInt(version);
+        }
+
+        /**
+         * Return the update number for an Oracle JVM.
+         * @return the update number
+         */
+        private int getUpdate()
+        {
+            String jreVersion = System.getProperty("java.version");
+            int updateSeparatorIndex = jreVersion.indexOf('_');
+            return Integer.parseInt(jreVersion.substring(updateSeparatorIndex + 1));
+        }
+
+        /**
+         * Checks if one of the specified options is being used.
+         * @param optionNames The name of the options to check
+         * @return {@code true} if one of the specified options is being used, {@code false} otherwise.
+         */
+        private boolean jvmOptionsContainsOneOf(String... optionNames)
+        {
+            RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
+            List<String> inputArguments = runtimeMxBean.getInputArguments();
+            for (String argument : inputArguments)
+            {
+                for (String optionName : optionNames)
+                    if (argument.startsWith(optionName))
+                        return true;
+            }
+            return false;
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 bfc8a0b..2d068de 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -19,11 +19,6 @@ package org.apache.cassandra.utils;
 
 import java.io.*;
 import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.text.StrBuilder;
@@ -32,7 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Utility to generate heap dumps.
+ * Utility to log heap histogram.
  *
  */
 public final class HeapUtils
@@ -43,54 +38,33 @@ public final class HeapUtils
      * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
      * or in the <code>CASSANDRA_HOME</code> directory.
      */
-    public static void generateHeapDump()
+    public static void logHeapHistogram()
     {
-        Long processId = getProcessId();
-        if (processId == null)
+        try
         {
-            logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-            return;
-        }
+            logger.info("Trying to log the heap histogram using jmap");
 
-        String heapDumpPath = getHeapDumpPathOption();
-        if (heapDumpPath == null)
-        {
-            String cassandraHome = System.getenv("CASSANDRA_HOME");
-            if (cassandraHome == null)
+            Long processId = getProcessId();
+            if (processId == null)
             {
+                logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                 return;
             }
 
-            heapDumpPath = cassandraHome;
-        }
+            String jmapPath = getJmapPath();
 
-        Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-        if (Files.isDirectory(dumpPath))
-        {
-            dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-        }
+            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
+            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
 
-        String jmapPath = getJmapPath();
+            String[] histoCommands = new String[] {jmapCommand,
+                    "-histo",
+                    processId.toString()};
 
-        // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-        String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
-
-        String[] dumpCommands = new String[] {jmapCommand,
-                                              "-dump:format=b,file=" + dumpPath,
-                                              processId.toString()};
-
-        // Lets also log the Heap histogram
-        String[] histoCommands = new String[] {jmapCommand,
-                                               "-histo",
-                                               processId.toString()};
-        try
-        {
-            logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
             logProcessOutput(Runtime.getRuntime().exec(histoCommands));
         }
-        catch (IOException e)
+        catch (Throwable e)
         {
-            logger.error("The heap dump could not be generated due to the following error: ", e);
+            logger.error("The heap histogram could not be generated due to the following error: ", e);
         }
     }
 
@@ -137,32 +111,6 @@ public final class HeapUtils
     }
 
     /**
-     * Retrieves the value of the <code>HeapDumpPath</code> JVM option.
-     * @return the value of the <code>HeapDumpPath</code> JVM option or <code>null</code> if the value has not been
-     * specified.
-     */
-    private static String getHeapDumpPathOption()
-    {
-        RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
-        List<String> inputArguments = runtimeMxBean.getInputArguments();
-        String heapDumpPathOption = null;
-        for (String argument : inputArguments)
-        {
-            if (argument.startsWith("-XX:HeapDumpPath="))
-            {
-                heapDumpPathOption = argument;
-                // We do not break in case the option has been specified several times.
-                // In general it seems that JVMs use the right-most argument as the winner.
-            }
-        }
-
-        if (heapDumpPathOption == null)
-            return null;
-
-        return heapDumpPathOption.substring(17, heapDumpPathOption.length());
-    }
-
-    /**
      * Retrieves the process ID or <code>null</code> if the process ID cannot be retrieved.
      * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index f8cb775..0196b04 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.net.SocketException;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,6 +39,8 @@ public final class JVMStabilityInspector
     private static final Logger logger = LoggerFactory.getLogger(JVMStabilityInspector.class);
     private static Killer killer = new Killer();
 
+    private static Object lock = new Object();
+    private static boolean printingHeapHistogram;
 
     private JVMStabilityInspector() {}
 
@@ -52,8 +55,25 @@ public final class JVMStabilityInspector
         boolean isUnstable = false;
         if (t instanceof OutOfMemoryError)
         {
-            isUnstable = true;
-            HeapUtils.generateHeapDump();
+            if (Boolean.getBoolean("cassandra.printHeapHistogramOnOutOfMemoryError"))
+            {
+                // We want to avoid printing multiple time the heap histogram if multiple OOM errors happen in a short
+                // time span.
+                synchronized(lock)
+                {
+                    if (printingHeapHistogram)
+                        return;
+                    printingHeapHistogram = true;
+                }
+                HeapUtils.logHeapHistogram();
+            }
+
+            logger.error("OutOfMemory error letting the JVM handle the error:", t);
+
+            StorageService.instance.removeShutdownHook();
+            // We let the JVM handle the error. The startup checks should have warned the user if it did not configure
+            // the JVM behavior in case of OOM (CASSANDRA-13006).
+            throw (OutOfMemoryError) t;
         }
 
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index 7142f97..f96ac6e 100644
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@ -20,14 +20,19 @@ package org.apache.cassandra.utils;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+
+import static java.util.Arrays.asList;
+
 import org.junit.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.SocketException;
+import java.util.Arrays;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class JVMStabilityInspectorTest
 {
@@ -45,10 +50,6 @@ public class JVMStabilityInspectorTest
             JVMStabilityInspector.inspectThrowable(new IOException());
             assertFalse(killerForTests.wasKilled());
 
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new OutOfMemoryError());
-            assertTrue(killerForTests.wasKilled());
-
             DatabaseDescriptor.setDiskFailurePolicy(Config.DiskFailurePolicy.die);
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new FSReadError(new IOException(), "blah"));
@@ -62,11 +63,6 @@ public class JVMStabilityInspectorTest
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new Exception(new IOException()));
             assertFalse(killerForTests.wasKilled());
-
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new Exception(new OutOfMemoryError()));
-            assertTrue(killerForTests.wasKilled());
-
         }
         finally
         {
@@ -77,6 +73,23 @@ public class JVMStabilityInspectorTest
     }
 
     @Test
+    public void testOutOfMemoryHandling()
+    {
+        for (Throwable oom : asList(new OutOfMemoryError(), new Exception(new OutOfMemoryError())))
+        {
+            try
+            {
+                JVMStabilityInspector.inspectThrowable(oom);
+                fail("The JVMStabilityInspector should delegate the handling of OutOfMemoryErrors to the JVM");
+            }
+            catch (OutOfMemoryError e)
+            {
+                assertTrue(true);
+            }
+        }
+    }
+
+    @Test
     public void fileHandleTest()
     {
         KillerForTests killerForTests = new KillerForTests();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[04/10] cassandra git commit: Rely on the JVM to handle OutOfMemoryErrors

Posted by bl...@apache.org.
Rely on the JVM to handle OutOfMemoryErrors

patch by Benjamin Lerer; reviewed by Joshua McKenzie for CASSANDRA-13006


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

Branch: refs/heads/trunk
Commit: 02aba7343ce300397ab672bbb1788aa8182d8a48
Parents: 3cd2c3c
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:21:05 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:21:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  9 ++-
 bin/cassandra                                   | 19 ++++-
 conf/cassandra-env.ps1                          | 10 +++
 conf/cassandra-env.sh                           | 12 +++
 .../apache/cassandra/service/StartupChecks.java | 74 ++++++++++++++++++
 .../org/apache/cassandra/utils/HeapUtils.java   | 82 ++++----------------
 .../cassandra/utils/JVMStabilityInspector.java  | 24 +++++-
 .../utils/JVMStabilityInspectorTest.java        | 31 +++++---
 9 files changed, 178 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1e81fd..5200eb1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.12
+ * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
  * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 
 2.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3bff458..5747941 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,8 +18,13 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.2 if you are upgrading
-      from a previous version.
+    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
+      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
+      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
+      options are used. See CASSANDRA-13006 for more details.
+    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
+      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
+      for more details.
 
 2.2.11
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/bin/cassandra
----------------------------------------------------------------------
diff --git a/bin/cassandra b/bin/cassandra
index 2dd0fe1..0e337e8 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -28,6 +28,7 @@
 #
 #   CLASSPATH -- A Java classpath containing everything necessary to run.
 #   JVM_OPTS -- Additional arguments to the JVM for heap size, etc
+#   JVM_ON_OUT_OF_MEMORY_ERROR_OPT -- The OnOutOfMemoryError JVM option if specified
 #   CASSANDRA_CONF -- Directory containing Cassandra configuration files.
 #
 # As a convenience, a fragment of shell is sourced in order to set one or
@@ -199,12 +200,22 @@ launch_service()
     # to close stdout/stderr, but it's up to us not to background.
     if [ "x$foreground" != "x" ]; then
         cassandra_parms="$cassandra_parms -Dcassandra-foreground=yes"
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class"
+        fi
     # Startup CassandraDaemon, background it, and write the pid.
     else
-        exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
-        [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
-        true
+        if [ "x$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" != "x" ]; then
+            exec $NUMACTL "$JAVA" $JVM_OPTS "$JVM_ON_OUT_OF_MEMORY_ERROR_OPT" $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        else
+            exec $NUMACTL "$JAVA" $JVM_OPTS $cassandra_parms -cp "$CLASSPATH" $props "$class" <&- &
+            [ ! -z "$pidpath" ] && printf "%d" $! > "$pidpath"
+            true
+        fi
     fi
 
     return $?

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 321a9ca..7b4a632 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -390,6 +390,16 @@ Function SetCassandraEnvironment
     $env:JVM_OPTS="$env:JVM_OPTS -Xmn$env:HEAP_NEWSIZE"
     $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 
+    # stop the jvm on OutOfMemoryError as it can result in some data corruption
+    # uncomment the preferred option
+    # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+    $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+
+    # print an heap histogram on OutOfMemoryError
+    # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
     # Per-thread stack size.
     $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index b519b76..7b1b8d3 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -204,6 +204,18 @@ fi
 
 startswith() { [ "${1#$2}" != "$1" ]; }
 
+# stop the jvm on OutOfMemoryError as it can result in some data corruption
+# uncomment the preferred option
+# For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+# on white spaces without taking quotes into account
+# ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+# JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+# JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+
+# print an heap histogram on OutOfMemoryError
+# JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+
 # Per-thread stack size.
 JVM_OPTS="$JVM_OPTS -Xss256k"
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 34bc824..7ec16d1 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.service;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
 import java.nio.file.*;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
@@ -178,6 +180,78 @@ public class StartupChecks
             {
                 logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
             }
+            else
+            {
+                    checkOutOfMemoryHandling();
+            }
+        }
+
+        /**
+         * Checks that the JVM is configured to handle OutOfMemoryError
+         */
+        private void checkOutOfMemoryHandling()
+        {
+            int version = getJavaVersion();
+            int update = getUpdate();
+            // The ExitOnOutOfMemory and CrashOnOutOfMemory are supported since the version 7u101 and 8u92
+            boolean jreSupportExitOnOutOfMemory = version > 8
+                                                    || (version == 7 && update >= 101)
+                                                    || (version == 8 && update >= 92);
+            if (jreSupportExitOnOutOfMemory)
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError=", "-XX:+ExitOnOutOfMemoryError", "-XX:+CrashOnOutOfMemoryError"))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                                + " Use one of the following JVM options to configure the behavior on OutOfMemoryError: "
+                                + " -XX:+ExitOnOutOfMemoryError, -XX:+CrashOnOutOfMemoryError, or -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\"");
+            }
+            else
+            {
+                if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError="))
+                    logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
+                            + " Either upgrade your JRE to a version greater or equal to 8u92 and use -XX:+ExitOnOutOfMemoryError/-XX:+CrashOnOutOfMemoryError"
+                            + " or use -XX:OnOutOfMemoryError=\"<cmd args>;<cmd args>\" on your current JRE.");
+            }
+        }
+
+        /**
+         * Returns the java version number for an Oracle JVM.
+         * @return the java version number
+         */
+        private int getJavaVersion()
+        {
+            String jreVersion = System.getProperty("java.version");
+            String version = jreVersion.startsWith("1.") ? jreVersion.substring(2, 3) // Pre 9 version
+                                                         : jreVersion.substring(0, jreVersion.indexOf('.'));
+            return Integer.parseInt(version);
+        }
+
+        /**
+         * Return the update number for an Oracle JVM.
+         * @return the update number
+         */
+        private int getUpdate()
+        {
+            String jreVersion = System.getProperty("java.version");
+            int updateSeparatorIndex = jreVersion.indexOf('_');
+            return Integer.parseInt(jreVersion.substring(updateSeparatorIndex + 1));
+        }
+
+        /**
+         * Checks if one of the specified options is being used.
+         * @param optionNames The name of the options to check
+         * @return {@code true} if one of the specified options is being used, {@code false} otherwise.
+         */
+        private boolean jvmOptionsContainsOneOf(String... optionNames)
+        {
+            RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
+            List<String> inputArguments = runtimeMxBean.getInputArguments();
+            for (String argument : inputArguments)
+            {
+                for (String optionName : optionNames)
+                    if (argument.startsWith(optionName))
+                        return true;
+            }
+            return false;
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/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 bfc8a0b..2d068de 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -19,11 +19,6 @@ package org.apache.cassandra.utils;
 
 import java.io.*;
 import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.text.StrBuilder;
@@ -32,7 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Utility to generate heap dumps.
+ * Utility to log heap histogram.
  *
  */
 public final class HeapUtils
@@ -43,54 +38,33 @@ public final class HeapUtils
      * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
      * or in the <code>CASSANDRA_HOME</code> directory.
      */
-    public static void generateHeapDump()
+    public static void logHeapHistogram()
     {
-        Long processId = getProcessId();
-        if (processId == null)
+        try
         {
-            logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
-            return;
-        }
+            logger.info("Trying to log the heap histogram using jmap");
 
-        String heapDumpPath = getHeapDumpPathOption();
-        if (heapDumpPath == null)
-        {
-            String cassandraHome = System.getenv("CASSANDRA_HOME");
-            if (cassandraHome == null)
+            Long processId = getProcessId();
+            if (processId == null)
             {
+                logger.error("The process ID could not be retrieved. Skipping heap histogram generation.");
                 return;
             }
 
-            heapDumpPath = cassandraHome;
-        }
+            String jmapPath = getJmapPath();
 
-        Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
-        if (Files.isDirectory(dumpPath))
-        {
-            dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
-        }
+            // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
+            String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
 
-        String jmapPath = getJmapPath();
+            String[] histoCommands = new String[] {jmapCommand,
+                    "-histo",
+                    processId.toString()};
 
-        // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
-        String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
-
-        String[] dumpCommands = new String[] {jmapCommand,
-                                              "-dump:format=b,file=" + dumpPath,
-                                              processId.toString()};
-
-        // Lets also log the Heap histogram
-        String[] histoCommands = new String[] {jmapCommand,
-                                               "-histo",
-                                               processId.toString()};
-        try
-        {
-            logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
             logProcessOutput(Runtime.getRuntime().exec(histoCommands));
         }
-        catch (IOException e)
+        catch (Throwable e)
         {
-            logger.error("The heap dump could not be generated due to the following error: ", e);
+            logger.error("The heap histogram could not be generated due to the following error: ", e);
         }
     }
 
@@ -137,32 +111,6 @@ public final class HeapUtils
     }
 
     /**
-     * Retrieves the value of the <code>HeapDumpPath</code> JVM option.
-     * @return the value of the <code>HeapDumpPath</code> JVM option or <code>null</code> if the value has not been
-     * specified.
-     */
-    private static String getHeapDumpPathOption()
-    {
-        RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
-        List<String> inputArguments = runtimeMxBean.getInputArguments();
-        String heapDumpPathOption = null;
-        for (String argument : inputArguments)
-        {
-            if (argument.startsWith("-XX:HeapDumpPath="))
-            {
-                heapDumpPathOption = argument;
-                // We do not break in case the option has been specified several times.
-                // In general it seems that JVMs use the right-most argument as the winner.
-            }
-        }
-
-        if (heapDumpPathOption == null)
-            return null;
-
-        return heapDumpPathOption.substring(17, heapDumpPathOption.length());
-    }
-
-    /**
      * Retrieves the process ID or <code>null</code> if the process ID cannot be retrieved.
      * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index f8cb775..0196b04 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.net.SocketException;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,6 +39,8 @@ public final class JVMStabilityInspector
     private static final Logger logger = LoggerFactory.getLogger(JVMStabilityInspector.class);
     private static Killer killer = new Killer();
 
+    private static Object lock = new Object();
+    private static boolean printingHeapHistogram;
 
     private JVMStabilityInspector() {}
 
@@ -52,8 +55,25 @@ public final class JVMStabilityInspector
         boolean isUnstable = false;
         if (t instanceof OutOfMemoryError)
         {
-            isUnstable = true;
-            HeapUtils.generateHeapDump();
+            if (Boolean.getBoolean("cassandra.printHeapHistogramOnOutOfMemoryError"))
+            {
+                // We want to avoid printing multiple time the heap histogram if multiple OOM errors happen in a short
+                // time span.
+                synchronized(lock)
+                {
+                    if (printingHeapHistogram)
+                        return;
+                    printingHeapHistogram = true;
+                }
+                HeapUtils.logHeapHistogram();
+            }
+
+            logger.error("OutOfMemory error letting the JVM handle the error:", t);
+
+            StorageService.instance.removeShutdownHook();
+            // We let the JVM handle the error. The startup checks should have warned the user if it did not configure
+            // the JVM behavior in case of OOM (CASSANDRA-13006).
+            throw (OutOfMemoryError) t;
         }
 
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02aba734/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index 7142f97..f96ac6e 100644
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@ -20,14 +20,19 @@ package org.apache.cassandra.utils;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
+
+import static java.util.Arrays.asList;
+
 import org.junit.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.SocketException;
+import java.util.Arrays;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class JVMStabilityInspectorTest
 {
@@ -45,10 +50,6 @@ public class JVMStabilityInspectorTest
             JVMStabilityInspector.inspectThrowable(new IOException());
             assertFalse(killerForTests.wasKilled());
 
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new OutOfMemoryError());
-            assertTrue(killerForTests.wasKilled());
-
             DatabaseDescriptor.setDiskFailurePolicy(Config.DiskFailurePolicy.die);
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new FSReadError(new IOException(), "blah"));
@@ -62,11 +63,6 @@ public class JVMStabilityInspectorTest
             killerForTests.reset();
             JVMStabilityInspector.inspectThrowable(new Exception(new IOException()));
             assertFalse(killerForTests.wasKilled());
-
-            killerForTests.reset();
-            JVMStabilityInspector.inspectThrowable(new Exception(new OutOfMemoryError()));
-            assertTrue(killerForTests.wasKilled());
-
         }
         finally
         {
@@ -77,6 +73,23 @@ public class JVMStabilityInspectorTest
     }
 
     @Test
+    public void testOutOfMemoryHandling()
+    {
+        for (Throwable oom : asList(new OutOfMemoryError(), new Exception(new OutOfMemoryError())))
+        {
+            try
+            {
+                JVMStabilityInspector.inspectThrowable(oom);
+                fail("The JVMStabilityInspector should delegate the handling of OutOfMemoryErrors to the JVM");
+            }
+            catch (OutOfMemoryError e)
+            {
+                assertTrue(true);
+            }
+        }
+    }
+
+    @Test
     public void fileHandleTest()
     {
         KillerForTests killerForTests = new KillerForTests();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[09/10] cassandra git commit: Merge branch cassandra-3.0 into cassandra-3.11

Posted by bl...@apache.org.
Merge branch cassandra-3.0 into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: a7c45be936bf8aa04b657529964c41b0d82c50e2
Parents: 88a41fb dd187d1
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Dec 12 10:38:33 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Dec 12 10:42:50 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 NEWS.txt                                        |   8 +-
 bin/cassandra                                   |  19 +-
 conf/cassandra-env.ps1                          | 914 ++++++++++---------
 conf/cassandra-env.sh                           |  12 +
 .../apache/cassandra/service/StartupChecks.java |  74 ++
 .../org/apache/cassandra/utils/HeapUtils.java   |  93 +-
 .../cassandra/utils/JVMStabilityInspector.java  |  24 +-
 .../utils/JVMStabilityInspectorTest.java        |  28 +-
 9 files changed, 632 insertions(+), 542 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 6e9a0bd,20ccc4b..116c7e9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,15 -1,4 +1,14 @@@
 -3.0.16
 +3.11.2
 + * Fix imbalanced disks when replacing node with same address with JBOD (CASSANDRA-14084)
 + * Reload compaction strategies when disk boundaries are invalidated (CASSANDRA-13948)
 + * Remove OpenJDK log warning (CASSANDRA-13916)
 + * Prevent compaction strategies from looping indefinitely (CASSANDRA-14079)
 + * Cache disk boundaries (CASSANDRA-13215)
 + * Add asm jar to build.xml for maven builds (CASSANDRA-11193)
 + * Round buffer size to powers of 2 for the chunk cache (CASSANDRA-13897)
 + * Update jackson JSON jars (CASSANDRA-13949)
 + * Avoid locks when checking LCS fanout and if we should defrag (CASSANDRA-13930)
 +Merged from 3.0:
- 3.0.16
   * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
   * Accept role names containing forward-slash (CASSANDRA-14088)
   * Optimize CRC check chance probability calculations (CASSANDRA-14094)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 9812b22,1bbe1aa..0c32278
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,7 -18,13 +18,13 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
-     - Nothing specific to this release, but please see previous upgrading sections.
 -   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 -     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 -     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 -     options are used. See CASSANDRA-13006 for more details.
 -   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 -     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 -     for more details.
++    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
++      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
++      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
++      options are used. See CASSANDRA-13006 for more details
++    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
++      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
++      for more details.
  
  Materialized Views
  -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/bin/cassandra
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --cc conf/cassandra-env.ps1
index 2df7054,405ed92..49d03ce
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@@ -393,70 -393,98 +393,80 @@@ Function SetCassandraEnvironmen
          $env:JVM_OPTS="$env:JVM_OPTS -XX:HeapDumpPath=$env:CASSANDRA_HEAPDUMP_DIR\cassandra-$unixTimestamp-pid$pid.hprof"
      }
  
-     if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
-     {
-         echo "Cassandra 3.0 and later require Java 8u40 or later."
-         exit
-     }
- 
-     # Specifies the default port over which Cassandra will be available for
-     # JMX connections.
-     $JMX_PORT="7199"
- 
-     # store in env to check if it's avail in verification
-     $env:JMX_PORT=$JMX_PORT
- 
-     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
-     # library path.
-     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
- 
-     # jmx: metrics and administration interface
-     #
-     # add this if you're having trouble connecting:
-     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
-     #
-     # see
-     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
-     # for more on configuring JMX through firewalls, etc. (Short version:
-     # get it working with no firewall first.)
-     #
-     # Due to potential security exploits, Cassandra ships with JMX accessible
-     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
-     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
-     #
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
-     #
-     # JMX SSL options
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
-     #
-     # JMX auth options
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-     ## Basic file based authn & authz
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.access.file=C:/jmxremote.access"
- 
-     ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
-     ## JAAS login modules can be used for authentication by uncommenting these two properties.
-     ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
-     ## which delegates to the IAuthenticator configured in cassandra.yaml
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
-     #$env:JVM_OPTS="$env:JVM_OPTS -Djava.security.auth.login.config=C:/cassandra-jaas.config"
- 
-     ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
-     ## uncomment this to use it. Requires one of the two authentication options to be enabled
-     #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
- 
-     # Default JMX setup, bound to local loopback address only
-     $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
- 
-     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
- }
 -    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
 -    {
 -        echo "Cassandra 3.0 and later require Java 8u40 or later."
 -        exit
 -    }
 -
 -    # enable assertions.  disabling this in production will give a modest
 -    # performance benefit (around 5%).
 -    $env:JVM_OPTS = "$env:JVM_OPTS -ea"
 -
 -    # Specifies the default port over which Cassandra will be available for
 -    # JMX connections.
 -    $JMX_PORT="7199"
 -
 -    # store in env to check if it's avail in verification
 -    $env:JMX_PORT=$JMX_PORT
 -
 -    # enable thread priorities, primarily so we can give periodic tasks
 -    # a lower priority to avoid interfering with client workload
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
 -    # allows lowering thread priority without being root on linux - probably
 -    # not necessary on Windows but doesn't harm anything.
 -    # see http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workar
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:ThreadPriorityPolicy=42"
 -
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError"
 -
+     # stop the jvm on OutOfMemoryError as it can result in some data corruption
+     # uncomment the preferred option
+     # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+     # $env:JVM_OPTS="$env:JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+     # $env:JVM_OPTS="$env:JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+     $env:JVM_OPTS="$env:JVM_OPTS -XX:OnOutOfMemoryError=""taskkill /F /PID %p"""
+ 
+     # print an heap histogram on OutOfMemoryError
+     # $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -    # Per-thread stack size.
 -    $env:JVM_OPTS="$env:JVM_OPTS -Xss256k"
 -
 -    # Larger interned string table, for gossip's benefit (CASSANDRA-6410)
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:StringTableSize=1000003"
 -
 -    # Make sure all memory is faulted and zeroed on startup.
 -    # This helps prevent soft faults in containers and makes
 -    # transparent hugepage allocation more effective.
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+AlwaysPreTouch"
 -
 -    # Biased locking does not benefit Cassandra.
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:-UseBiasedLocking"
++    if ($env:JVM_VERSION.CompareTo("1.8.0") -eq -1 -or [convert]::ToInt32($env:JVM_PATCH_VERSION) -lt 40)
++    {
++        echo "Cassandra 3.0 and later require Java 8u40 or later."
++        exit
++    }
+ 
 -    # Enable thread-local allocation blocks and allow the JVM to automatically
 -    # resize them at runtime.
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB -XX:+ResizeTLAB"
++    # Specifies the default port over which Cassandra will be available for
++    # JMX connections.
++    $JMX_PORT="7199"
+ 
 -    # http://www.evanjones.ca/jvm-mmap-pause.html
 -    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"
++    # store in env to check if it's avail in verification
++    $env:JMX_PORT=$JMX_PORT
+ 
+     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system
+     # library path.
+     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
+     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
+ 
 -    # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -    # $env:JVM_OPTS="$env:JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -    # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -    # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -    # comment out this entry to enable IPv6 support).
 -    $env:JVM_OPTS="$env:JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
+     # jmx: metrics and administration interface
+     #
+     # add this if you're having trouble connecting:
+     # $env:JVM_OPTS="$env:JVM_OPTS -Djava.rmi.server.hostname=<public name>"
+     #
+     # see
+     # https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
+     # for more on configuring JMX through firewalls, etc. (Short version:
+     # get it working with no firewall first.)
+     #
+     # Due to potential security exploits, Cassandra ships with JMX accessible
+     # *only* from localhost.  To enable remote JMX connections, uncomment lines below
+     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
+     #
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
 -    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
++    #
++    # JMX SSL options
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
++    #
++    # JMX auth options
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
++    ## Basic file based authn & authz
+     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
 -    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.access.file=C:/jmxremote.access"
+ 
 -    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
++    ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
++    ## JAAS login modules can be used for authentication by uncommenting these two properties.
++    ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
++    ## which delegates to the IAuthenticator configured in cassandra.yaml
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
++    #$env:JVM_OPTS="$env:JVM_OPTS -Djava.security.auth.login.config=C:/cassandra-jaas.config"
++
++    ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
++    ## uncomment this to use it. Requires one of the two authentication options to be enabled
++    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
+ 
 -    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
++    # Default JMX setup, bound to local loopback address only
++    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
++
++    $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --cc conf/cassandra-env.sh
index 5a02f79,ffc773f..189576b
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@@ -214,6 -247,30 +214,18 @@@ if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" 
      JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
  fi
  
+ # stop the jvm on OutOfMemoryError as it can result in some data corruption
+ # uncomment the preferred option
+ # ExitOnOutOfMemoryError and CrashOnOutOfMemoryError require a JRE greater or equals to 1.7 update 101 or 1.8 update 92
+ # For OnOutOfMemoryError we cannot use the JVM_OPTS variables because bash commands split words
+ # on white spaces without taking quotes into account
+ # JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
+ # JVM_OPTS="$JVM_OPTS -XX:+CrashOnOutOfMemoryError"
+ JVM_ON_OUT_OF_MEMORY_ERROR_OPT="-XX:OnOutOfMemoryError=kill -9 %p"
+ 
+ # print an heap histogram on OutOfMemoryError
+ # JVM_OPTS="$JVM_OPTS -Dcassandra.printHeapHistogramOnOutOfMemoryError=true"
+ 
 -# uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 -# JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 -
 -# uncomment to have Cassandra JVM log internal method compilation (developers only)
 -# JVM_OPTS="$JVM_OPTS -XX:+UnlockDiagnosticVMOptions -XX:+LogCompilation"
 -# JVM_OPTS="$JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 -
 -# Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 -# http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
 -# comment out this entry to enable IPv6 support).
 -JVM_OPTS="$JVM_OPTS -Djava.net.preferIPv4Stack=true"
 -
  # jmx: metrics and administration interface
  #
  # add this if you're having trouble connecting:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/utils/HeapUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7c45be9/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org