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/08/08 15:16:54 UTC

[01/10] cassandra git commit: Prevent integer overflow on exabyte filesystems

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 739cd2bc2 -> 270f690ff
  refs/heads/cassandra-3.0 396026047 -> 1a70dede3
  refs/heads/cassandra-3.11 47a2839bf -> 303dba650
  refs/heads/trunk cad941653 -> c00206297


Prevent integer overflow on exabyte filesystems

patch by Matt Wringe and Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13067


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

Branch: refs/heads/cassandra-2.2
Commit: 270f690ff6047cc3e797a3f34b7efa26e7232183
Parents: 739cd2b
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:51:03 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 16:51:03 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 182 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 36c34a1..f712333 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.11
+ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
  * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
  * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
  * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 981026d..90a82fe 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.IOException;
 import java.net.*;
 import java.nio.file.FileStore;
-import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -56,6 +55,9 @@ import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.memory.*;
 
+import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
+import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+
 public class DatabaseDescriptor
 {
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -530,7 +532,7 @@ public class DatabaseDescriptor
             try
             {
                 // use 1/4 of available space.  See discussion on #10013 and #10199
-                minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
+                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
             }
             catch (IOException e)
             {
@@ -576,7 +578,7 @@ public class DatabaseDescriptor
 
             try
             {
-                dataFreeBytes += guessFileStore(datadir).getUnallocatedSpace();
+                dataFreeBytes = saturatedSum(dataFreeBytes, guessFileStore(datadir).getUnallocatedSpace());
             }
             catch (IOException e)
             {
@@ -585,9 +587,9 @@ public class DatabaseDescriptor
                                                                datadir), e);
             }
         }
-        if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
+        if (dataFreeBytes < 64 * ONE_GB)
             logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
-                        dataFreeBytes / 1048576);
+                        dataFreeBytes / ONE_MB);
 
 
         if (conf.commitlog_directory.equals(conf.saved_caches_directory))
@@ -697,6 +699,20 @@ public class DatabaseDescriptor
             throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
     }
 
+    /**
+     * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+     *
+     * @param left the left operand
+     * @param right the right operand
+     * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+     */
+    private static long saturatedSum(long left, long right)
+    {
+        assert left >= 0 && right >= 0;
+        long sum = left + right;
+        return sum < 0 ? Long.MAX_VALUE : sum;
+    }
+
     private static FileStore guessFileStore(String dir) throws IOException
     {
         Path path = Paths.get(dir);
@@ -704,7 +720,7 @@ public class DatabaseDescriptor
         {
             try
             {
-                return Files.getFileStore(path);
+                return FileUtils.getFileStore(path);
             }
             catch (IOException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 2b3662f..fa76b61 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -482,7 +482,7 @@ public class Directories
 
         public long getAvailableSpace()
         {
-            return location.getUsableSpace();
+            return FileUtils.getUsableSpace(location);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 8d122dd..bf0fae5 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -21,11 +21,12 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.*;
+import java.nio.file.attribute.FileAttributeView;
+import java.nio.file.attribute.FileStoreAttributeView;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.cassandra.config.Config;
 import sun.nio.ch.DirectBuffer;
 
 import org.slf4j.Logger;
@@ -45,10 +46,10 @@ import static org.apache.cassandra.utils.Throwables.merge;
 public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-    private static final double KB = 1024d;
-    private static final double MB = 1024*1024d;
-    private static final double GB = 1024*1024*1024d;
-    private static final double TB = 1024*1024*1024*1024d;
+    public static final long ONE_KB = 1024;
+    public static final long ONE_MB = 1024 * ONE_KB;
+    public static final long ONE_GB = 1024 * ONE_MB;
+    public static final long ONE_TB = 1024 * ONE_GB;
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
@@ -330,27 +331,27 @@ public final class FileUtils
     public static String stringifyFileSize(double value)
     {
         double d;
-        if ( value >= TB )
+        if ( value >= ONE_TB )
         {
-            d = value / TB;
+            d = value / ONE_TB;
             String val = df.format(d);
             return val + " TB";
         }
-        else if ( value >= GB )
+        else if ( value >= ONE_GB )
         {
-            d = value / GB;
+            d = value / ONE_GB;
             String val = df.format(d);
             return val + " GB";
         }
-        else if ( value >= MB )
+        else if ( value >= ONE_MB )
         {
-            d = value / MB;
+            d = value / ONE_MB;
             String val = df.format(d);
             return val + " MB";
         }
-        else if ( value >= KB )
+        else if ( value >= ONE_KB )
         {
-            d = value / KB;
+            d = value / ONE_KB;
             String val = df.format(d);
             return val + " KB";
         }
@@ -478,4 +479,159 @@ public final class FileUtils
     {
         fsErrorHandler.getAndSet(handler);
     }
+
+    /**
+     * Returns the size of the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+     * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+     *
+     * @param file the partition
+     * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+     */
+    public static long getTotalSpace(File file)
+    {
+        return handleLargeFileSystem(file.getTotalSpace());
+    }
+
+    /**
+     * Returns the number of unallocated bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of unallocated bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getFreeSpace(File file)
+    {
+        return handleLargeFileSystem(file.getFreeSpace());
+    }
+
+    /**
+     * Returns the number of available bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of available bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getUsableSpace(File file)
+    {
+        return handleLargeFileSystem(file.getUsableSpace());
+    }
+
+    /**
+     * Returns the {@link FileStore} representing the file store where a file
+     * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+     * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+     * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+     * for more information.
+     *
+     * @param path the path to the file
+     * @return the file store where the file is stored
+     */
+    public static FileStore getFileStore(Path path) throws IOException
+    {
+        return new SafeFileStore(Files.getFileStore(path));
+    }
+
+    /**
+     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+     * @param size returned by the Java's FileStore methods
+     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     */
+    private static long handleLargeFileSystem(long size)
+    {
+        return size < 0 ? Long.MAX_VALUE : size;
+    }
+
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private FileUtils()
+    {
+    }
+
+    /**
+     * FileStore decorator used to safely handle large file system.
+     *
+     * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+     * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+     * return {@code Long.MAX_VALUE} if the size overflow.</p>
+     *
+     * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+     */
+    private static final class SafeFileStore extends FileStore
+    {
+        /**
+         * The decorated {@code FileStore}
+         */
+        private final FileStore fileStore;
+
+        public SafeFileStore(FileStore fileStore)
+        {
+            this.fileStore = fileStore;
+        }
+
+        @Override
+        public String name()
+        {
+            return fileStore.name();
+        }
+
+        @Override
+        public String type()
+        {
+            return fileStore.type();
+        }
+
+        @Override
+        public boolean isReadOnly()
+        {
+            return fileStore.isReadOnly();
+        }
+
+        @Override
+        public long getTotalSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getTotalSpace());
+        }
+
+        @Override
+        public long getUsableSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUsableSpace());
+        }
+
+        @Override
+        public long getUnallocatedSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+        {
+            return fileStore.supportsFileAttributeView(type);
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(String name)
+        {
+            return fileStore.supportsFileAttributeView(name);
+        }
+
+        @Override
+        public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+        {
+            return fileStore.getFileStoreAttributeView(type);
+        }
+
+        @Override
+        public Object getAttribute(String attribute) throws IOException
+        {
+            return fileStore.getAttribute(attribute);
+        }
+    }
 }


---------------------------------------------------------------------
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/303dba65
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/303dba65
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/303dba65

Branch: refs/heads/trunk
Commit: 303dba6504ba069b5ea92ee18a47d3ba87c1563e
Parents: 47a2839 1a70ded
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 17:09:06 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:11:24 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  26 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 191 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index b778df6,1f42c70..145a746
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -17,9 -13,11 +17,10 @@@ Merged from 3.0
   * 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)
 - Merged from 2.2:
 +Merged from 2.2:
+  * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067)
   * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
   * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
   * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 87b388e,aba7617..ad43565
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -19,10 -19,8 +19,9 @@@ package org.apache.cassandra.config
  
  import java.io.File;
  import java.io.IOException;
 +import java.lang.reflect.Constructor;
  import java.net.*;
  import java.nio.file.FileStore;
- import java.nio.file.Files;
  import java.nio.file.NoSuchFileException;
  import java.nio.file.Path;
  import java.nio.file.Paths;
@@@ -46,25 -40,24 +45,27 @@@ import org.apache.cassandra.config.Conf
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.io.FSWriteError;
 -import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.util.DiskOptimizationStrategy;
  import org.apache.cassandra.io.util.FileUtils;
 -import org.apache.cassandra.locator.*;
 -import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy;
 +import org.apache.cassandra.io.util.SsdDiskOptimizationStrategy;
 +import org.apache.cassandra.locator.DynamicEndpointSnitch;
 +import org.apache.cassandra.locator.EndpointSnitchInfo;
 +import org.apache.cassandra.locator.IEndpointSnitch;
 +import org.apache.cassandra.locator.SeedProvider;
 +import org.apache.cassandra.net.BackPressureStrategy;
 +import org.apache.cassandra.net.RateBasedBackPressure;
  import org.apache.cassandra.scheduler.IRequestScheduler;
  import org.apache.cassandra.scheduler.NoScheduler;
 -import org.apache.cassandra.service.CacheService;
 -import org.apache.cassandra.thrift.ThriftServer;
 +import org.apache.cassandra.security.EncryptionContext;
 +import org.apache.cassandra.service.CacheService.CacheType;
 +import org.apache.cassandra.thrift.ThriftServer.ThriftServerType;
  import org.apache.cassandra.utils.FBUtilities;
 -import org.apache.cassandra.utils.memory.*;
 +
 +import org.apache.commons.lang3.StringUtils;
  
+ import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
 -import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+ 
  public class DatabaseDescriptor
  {
      private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@@ -429,53 -443,96 +430,53 @@@
          if (conf.native_transport_max_frame_size_in_mb <= 0)
              throw new ConfigurationException("native_transport_max_frame_size_in_mb must be positive, but was " + conf.native_transport_max_frame_size_in_mb, false);
  
 -        // fail early instead of OOMing (see CASSANDRA-8116)
 -        if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads == Integer.MAX_VALUE)
 -            throw new ConfigurationException("The hsha rpc_server_type is not compatible with an rpc_max_threads " +
 -                                             "setting of 'unlimited'.  Please see the comments in cassandra.yaml " +
 -                                             "for rpc_server_type and rpc_max_threads.",
 -                                             false);
 -        if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads > (FBUtilities.getAvailableProcessors() * 2 + 1024))
 -            logger.warn("rpc_max_threads setting of {} may be too high for the hsha server and cause unnecessary thread contention, reducing performance", conf.rpc_max_threads);
 +        // if data dirs, commitlog dir, or saved caches dir are set in cassandra.yaml, use that.  Otherwise,
 +        // use -Dcassandra.storagedir (set in cassandra-env.sh) as the parent dir for data/, commitlog/, and saved_caches/
 +        if (conf.commitlog_directory == null)
 +        {
 +            conf.commitlog_directory = storagedirFor("commitlog");
 +        }
  
 -        /* end point snitch */
 -        if (conf.endpoint_snitch == null)
 +        if (conf.hints_directory == null)
          {
 -            throw new ConfigurationException("Missing endpoint_snitch directive", false);
 +            conf.hints_directory = storagedirFor("hints");
          }
 -        snitch = createEndpointSnitch(conf.endpoint_snitch);
 -        EndpointSnitchInfo.create();
  
 -        localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
 -        localComparator = new Comparator<InetAddress>()
 +        if (conf.cdc_raw_directory == null)
          {
 -            public int compare(InetAddress endpoint1, InetAddress endpoint2)
 -            {
 -                boolean local1 = localDC.equals(snitch.getDatacenter(endpoint1));
 -                boolean local2 = localDC.equals(snitch.getDatacenter(endpoint2));
 -                if (local1 && !local2)
 -                    return -1;
 -                if (local2 && !local1)
 -                    return 1;
 -                return 0;
 -            }
 -        };
 +            conf.cdc_raw_directory = storagedirFor("cdc_raw");
 +        }
  
 -        /* Request Scheduler setup */
 -        requestSchedulerOptions = conf.request_scheduler_options;
 -        if (conf.request_scheduler != null)
 +        if (conf.commitlog_total_space_in_mb == null)
          {
 +            int preferredSize = 8192;
 +            int minSize = 0;
              try
              {
 -                if (requestSchedulerOptions == null)
 -                {
 -                    requestSchedulerOptions = new RequestSchedulerOptions();
 -                }
 -                Class<?> cls = Class.forName(conf.request_scheduler);
 -                requestScheduler = (IRequestScheduler) cls.getConstructor(RequestSchedulerOptions.class).newInstance(requestSchedulerOptions);
 +                // use 1/4 of available space.  See discussion on #10013 and #10199
-                 minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
++                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
              }
 -            catch (ClassNotFoundException e)
 +            catch (IOException e)
              {
 -                throw new ConfigurationException("Invalid Request Scheduler class " + conf.request_scheduler, false);
 +                logger.debug("Error checking disk space", e);
 +                throw new ConfigurationException(String.format("Unable to check disk space available to %s. Perhaps the Cassandra user does not have the necessary permissions",
 +                                                               conf.commitlog_directory), e);
              }
 -            catch (Exception e)
 +            if (minSize < preferredSize)
              {
 -                throw new ConfigurationException("Unable to instantiate request scheduler", e);
 +                logger.warn("Small commitlog volume detected at {}; setting commitlog_total_space_in_mb to {}.  You can override this in cassandra.yaml",
 +                            conf.commitlog_directory, minSize);
 +                conf.commitlog_total_space_in_mb = minSize;
 +            }
 +            else
 +            {
 +                conf.commitlog_total_space_in_mb = preferredSize;
              }
 -        }
 -        else
 -        {
 -            requestScheduler = new NoScheduler();
          }
  
 -        if (conf.request_scheduler_id == RequestSchedulerId.keyspace)
 -        {
 -            requestSchedulerId = conf.request_scheduler_id;
 -        }
 -        else
 -        {
 -            // Default to Keyspace
 -            requestSchedulerId = RequestSchedulerId.keyspace;
 -        }
 -
 -        // if data dirs, commitlog dir, or saved caches dir are set in cassandra.yaml, use that.  Otherwise,
 -        // use -Dcassandra.storagedir (set in cassandra-env.sh) as the parent dir for data/, commitlog/, and saved_caches/
 -        if (conf.commitlog_directory == null)
 -        {
 -            conf.commitlog_directory = System.getProperty("cassandra.storagedir", null);
 -            if (conf.commitlog_directory == null)
 -                throw new ConfigurationException("commitlog_directory is missing and -Dcassandra.storagedir is not set", false);
 -            conf.commitlog_directory += File.separator + "commitlog";
 -        }
 -
 -        if (conf.hints_directory == null)
 +        if (conf.cdc_total_space_in_mb == 0)
          {
 -            conf.hints_directory = System.getProperty("cassandra.storagedir", null);
 -            if (conf.hints_directory == null)
 -                throw new ConfigurationException("hints_directory is missing and -Dcassandra.storagedir is not set", false);
 -            conf.hints_directory += File.separator + "hints";
 -        }
 -
 -        if (conf.commitlog_total_space_in_mb == null)
 -        {
 -            int preferredSize = 8192;
 +            int preferredSize = 4096;
              int minSize = 0;
              try
              {
@@@ -536,11 -594,11 +537,10 @@@
                                                                 datadir), e);
              }
          }
-         if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
 -        if (dataFreeBytes < 64 * ONE_GB)
 -            logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
 -                        dataFreeBytes / ONE_MB);
 -
++        if (dataFreeBytes < 64 * ONE_GB) // 64 GB
 +            logger.warn("Only {} free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
 +                        FBUtilities.prettyPrintMemory(dataFreeBytes));
  
- 
          if (conf.commitlog_directory.equals(conf.saved_caches_directory))
              throw new ConfigurationException("saved_caches_directory must not be the same as the commitlog_directory", false);
          if (conf.commitlog_directory.equals(conf.hints_directory))
@@@ -970,16 -699,62 +970,30 @@@
          }
          try
          {
 -            Class<?> seedProviderClass = Class.forName(conf.seed_provider.class_name);
 -            seedProvider = (SeedProvider)seedProviderClass.getConstructor(Map.class).newInstance(conf.seed_provider.parameters);
 +            partitioner = FBUtilities.newPartitioner(System.getProperty(Config.PROPERTY_PREFIX + "partitioner", conf.partitioner));
          }
 -        // there are about 5 checked exceptions that could be thrown here.
          catch (Exception e)
          {
 -            throw new ConfigurationException(e.getMessage() + "\nFatal configuration error; unable to start server.  See log for stacktrace.", true);
 -        }
 -        if (seedProvider.getSeeds().size() == 0)
 -            throw new ConfigurationException("The seed provider lists no seeds.", false);
 -
 -        if (conf.user_defined_function_fail_timeout < 0)
 -            throw new ConfigurationException("user_defined_function_fail_timeout must not be negative", false);
 -        if (conf.user_defined_function_warn_timeout < 0)
 -            throw new ConfigurationException("user_defined_function_warn_timeout must not be negative", false);
 -
 -        if (conf.user_defined_function_fail_timeout < conf.user_defined_function_warn_timeout)
 -            throw new ConfigurationException("user_defined_function_warn_timeout must less than user_defined_function_fail_timeout", false);
 -
 -        if (conf.max_mutation_size_in_kb == null)
 -            conf.max_mutation_size_in_kb = conf.commitlog_segment_size_in_mb * 1024 / 2;
 -        else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_in_kb)
 -            throw new ConfigurationException("commitlog_segment_size_in_mb must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 -
 -        // native transport encryption options
 -        if (conf.native_transport_port_ssl != null
 -            && conf.native_transport_port_ssl.intValue() != conf.native_transport_port.intValue()
 -            && !conf.client_encryption_options.enabled)
 -        {
 -            throw new ConfigurationException("Encryption must be enabled in client_encryption_options for native_transport_port_ssl", false);
 +            throw new ConfigurationException("Invalid partitioner class " + conf.partitioner, false);
          }
  
 -        if (conf.max_value_size_in_mb == null || conf.max_value_size_in_mb <= 0)
 -            throw new ConfigurationException("max_value_size_in_mb must be positive", false);
 -
 -        if (conf.otc_coalescing_enough_coalesced_messages > 128)
 -            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be smaller than 128", false);
 -
 -        if (conf.otc_coalescing_enough_coalesced_messages <= 0)
 -            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
 +        paritionerName = partitioner.getClass().getCanonicalName();
      }
  
+     /**
+      * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+      *
+      * @param left the left operand
+      * @param right the right operand
+      * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+      */
+     private static long saturatedSum(long left, long right)
+     {
+         assert left >= 0 && right >= 0;
+         long sum = left + right;
+         return sum < 0 ? Long.MAX_VALUE : sum;
+     }
+ 
      private static FileStore guessFileStore(String dir) throws IOException
      {
          Path path = Paths.get(dir);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index c6e4e63,80df67b..24017cf
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -23,7 -23,8 +23,9 @@@ import java.nio.channels.FileChannel
  import java.nio.charset.Charset;
  import java.nio.charset.StandardCharsets;
  import java.nio.file.*;
 +import java.nio.file.attribute.BasicFileAttributes;
+ import java.nio.file.attribute.FileAttributeView;
+ import java.nio.file.attribute.FileStoreAttributeView;
  import java.text.DecimalFormat;
  import java.util.Arrays;
  import java.util.Collections;
@@@ -54,14 -54,14 +56,14 @@@ public final class FileUtil
      public static final Charset CHARSET = StandardCharsets.UTF_8;
  
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-     private static final double KB = 1024d;
-     private static final double MB = 1024*1024d;
-     private static final double GB = 1024*1024*1024d;
-     private static final double TB = 1024*1024*1024*1024d;
+     public static final long ONE_KB = 1024;
+     public static final long ONE_MB = 1024 * ONE_KB;
+     public static final long ONE_GB = 1024 * ONE_MB;
+     public static final long ONE_TB = 1024 * ONE_GB;
  
      private static final DecimalFormat df = new DecimalFormat("#.##");
 -    private static final boolean canCleanDirectBuffers;
 -    private static final AtomicReference<FSErrorHandler> fsErrorHandler = new AtomicReference<>();
 +    public static final boolean isCleanerAvailable;
 +    private static final AtomicReference<Optional<FSErrorHandler>> fsErrorHandler = new AtomicReference<>(Optional.empty());
  
      static
      {
@@@ -415,29 -418,29 +417,29 @@@
      public static String stringifyFileSize(double value)
      {
          double d;
-         if ( value >= TB )
+         if ( value >= ONE_TB )
          {
-             d = value / TB;
+             d = value / ONE_TB;
              String val = df.format(d);
 -            return val + " TB";
 +            return val + " TiB";
          }
-         else if ( value >= GB )
+         else if ( value >= ONE_GB )
          {
-             d = value / GB;
+             d = value / ONE_GB;
              String val = df.format(d);
 -            return val + " GB";
 +            return val + " GiB";
          }
-         else if ( value >= MB )
+         else if ( value >= ONE_MB )
          {
-             d = value / MB;
+             d = value / ONE_MB;
              String val = df.format(d);
 -            return val + " MB";
 +            return val + " MiB";
          }
-         else if ( value >= KB )
+         else if ( value >= ONE_KB )
          {
-             d = value / KB;
+             d = value / ONE_KB;
              String val = df.format(d);
 -            return val + " KB";
 +            return val + " KiB";
          }
          else
          {
@@@ -606,6 -603,161 +608,161 @@@
  
      public static void setFSErrorHandler(FSErrorHandler handler)
      {
 -        fsErrorHandler.getAndSet(handler);
 +        fsErrorHandler.getAndSet(Optional.ofNullable(handler));
      }
+ 
+     /**
+      * Returns the size of the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+      * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+      *
+      * @param file the partition
+      * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+      */
+     public static long getTotalSpace(File file)
+     {
+         return handleLargeFileSystem(file.getTotalSpace());
+     }
+ 
+     /**
+      * Returns the number of unallocated bytes on the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+      * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+      *
+      * @param file the partition
+      * @return the number of unallocated bytes on the partition or {@code 0L}
+      * if the abstract pathname does not name a partition.
+      */
+     public static long getFreeSpace(File file)
+     {
+         return handleLargeFileSystem(file.getFreeSpace());
+     }
+ 
+     /**
+      * Returns the number of available bytes on the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+      * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+      *
+      * @param file the partition
+      * @return the number of available bytes on the partition or {@code 0L}
+      * if the abstract pathname does not name a partition.
+      */
+     public static long getUsableSpace(File file)
+     {
+         return handleLargeFileSystem(file.getUsableSpace());
+     }
+ 
+     /**
+      * Returns the {@link FileStore} representing the file store where a file
+      * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+      * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+      * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+      * for more information.
+      *
+      * @param path the path to the file
+      * @return the file store where the file is stored
+      */
+     public static FileStore getFileStore(Path path) throws IOException
+     {
+         return new SafeFileStore(Files.getFileStore(path));
+     }
+ 
+     /**
+      * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+      * @param size returned by the Java's FileStore methods
+      * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+      */
+     private static long handleLargeFileSystem(long size)
+     {
+         return size < 0 ? Long.MAX_VALUE : size;
+     }
+ 
+     /**
+      * Private constructor as the class contains only static methods.
+      */
+     private FileUtils()
+     {
+     }
+ 
+     /**
+      * FileStore decorator used to safely handle large file system.
+      *
+      * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+      * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+      * return {@code Long.MAX_VALUE} if the size overflow.</p>
+      *
+      * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+      */
+     private static final class SafeFileStore extends FileStore
+     {
+         /**
+          * The decorated {@code FileStore}
+          */
+         private final FileStore fileStore;
+ 
+         public SafeFileStore(FileStore fileStore)
+         {
+             this.fileStore = fileStore;
+         }
+ 
+         @Override
+         public String name()
+         {
+             return fileStore.name();
+         }
+ 
+         @Override
+         public String type()
+         {
+             return fileStore.type();
+         }
+ 
+         @Override
+         public boolean isReadOnly()
+         {
+             return fileStore.isReadOnly();
+         }
+ 
+         @Override
+         public long getTotalSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getTotalSpace());
+         }
+ 
+         @Override
+         public long getUsableSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getUsableSpace());
+         }
+ 
+         @Override
+         public long getUnallocatedSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+         }
+ 
+         @Override
+         public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+         {
+             return fileStore.supportsFileAttributeView(type);
+         }
+ 
+         @Override
+         public boolean supportsFileAttributeView(String name)
+         {
+             return fileStore.supportsFileAttributeView(name);
+         }
+ 
+         @Override
+         public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+         {
+             return fileStore.getFileStoreAttributeView(type);
+         }
+ 
+         @Override
+         public Object getAttribute(String attribute) throws IOException
+         {
+             return fileStore.getAttribute(attribute);
+         }
+     }
  }


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


[03/10] cassandra git commit: Prevent integer overflow on exabyte filesystems

Posted by bl...@apache.org.
Prevent integer overflow on exabyte filesystems

patch by Matt Wringe and Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13067


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

Branch: refs/heads/cassandra-3.11
Commit: 270f690ff6047cc3e797a3f34b7efa26e7232183
Parents: 739cd2b
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:51:03 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 16:51:03 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 182 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 36c34a1..f712333 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.11
+ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
  * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
  * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
  * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 981026d..90a82fe 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.IOException;
 import java.net.*;
 import java.nio.file.FileStore;
-import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -56,6 +55,9 @@ import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.memory.*;
 
+import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
+import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+
 public class DatabaseDescriptor
 {
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -530,7 +532,7 @@ public class DatabaseDescriptor
             try
             {
                 // use 1/4 of available space.  See discussion on #10013 and #10199
-                minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
+                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
             }
             catch (IOException e)
             {
@@ -576,7 +578,7 @@ public class DatabaseDescriptor
 
             try
             {
-                dataFreeBytes += guessFileStore(datadir).getUnallocatedSpace();
+                dataFreeBytes = saturatedSum(dataFreeBytes, guessFileStore(datadir).getUnallocatedSpace());
             }
             catch (IOException e)
             {
@@ -585,9 +587,9 @@ public class DatabaseDescriptor
                                                                datadir), e);
             }
         }
-        if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
+        if (dataFreeBytes < 64 * ONE_GB)
             logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
-                        dataFreeBytes / 1048576);
+                        dataFreeBytes / ONE_MB);
 
 
         if (conf.commitlog_directory.equals(conf.saved_caches_directory))
@@ -697,6 +699,20 @@ public class DatabaseDescriptor
             throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
     }
 
+    /**
+     * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+     *
+     * @param left the left operand
+     * @param right the right operand
+     * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+     */
+    private static long saturatedSum(long left, long right)
+    {
+        assert left >= 0 && right >= 0;
+        long sum = left + right;
+        return sum < 0 ? Long.MAX_VALUE : sum;
+    }
+
     private static FileStore guessFileStore(String dir) throws IOException
     {
         Path path = Paths.get(dir);
@@ -704,7 +720,7 @@ public class DatabaseDescriptor
         {
             try
             {
-                return Files.getFileStore(path);
+                return FileUtils.getFileStore(path);
             }
             catch (IOException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 2b3662f..fa76b61 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -482,7 +482,7 @@ public class Directories
 
         public long getAvailableSpace()
         {
-            return location.getUsableSpace();
+            return FileUtils.getUsableSpace(location);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 8d122dd..bf0fae5 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -21,11 +21,12 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.*;
+import java.nio.file.attribute.FileAttributeView;
+import java.nio.file.attribute.FileStoreAttributeView;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.cassandra.config.Config;
 import sun.nio.ch.DirectBuffer;
 
 import org.slf4j.Logger;
@@ -45,10 +46,10 @@ import static org.apache.cassandra.utils.Throwables.merge;
 public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-    private static final double KB = 1024d;
-    private static final double MB = 1024*1024d;
-    private static final double GB = 1024*1024*1024d;
-    private static final double TB = 1024*1024*1024*1024d;
+    public static final long ONE_KB = 1024;
+    public static final long ONE_MB = 1024 * ONE_KB;
+    public static final long ONE_GB = 1024 * ONE_MB;
+    public static final long ONE_TB = 1024 * ONE_GB;
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
@@ -330,27 +331,27 @@ public final class FileUtils
     public static String stringifyFileSize(double value)
     {
         double d;
-        if ( value >= TB )
+        if ( value >= ONE_TB )
         {
-            d = value / TB;
+            d = value / ONE_TB;
             String val = df.format(d);
             return val + " TB";
         }
-        else if ( value >= GB )
+        else if ( value >= ONE_GB )
         {
-            d = value / GB;
+            d = value / ONE_GB;
             String val = df.format(d);
             return val + " GB";
         }
-        else if ( value >= MB )
+        else if ( value >= ONE_MB )
         {
-            d = value / MB;
+            d = value / ONE_MB;
             String val = df.format(d);
             return val + " MB";
         }
-        else if ( value >= KB )
+        else if ( value >= ONE_KB )
         {
-            d = value / KB;
+            d = value / ONE_KB;
             String val = df.format(d);
             return val + " KB";
         }
@@ -478,4 +479,159 @@ public final class FileUtils
     {
         fsErrorHandler.getAndSet(handler);
     }
+
+    /**
+     * Returns the size of the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+     * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+     *
+     * @param file the partition
+     * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+     */
+    public static long getTotalSpace(File file)
+    {
+        return handleLargeFileSystem(file.getTotalSpace());
+    }
+
+    /**
+     * Returns the number of unallocated bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of unallocated bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getFreeSpace(File file)
+    {
+        return handleLargeFileSystem(file.getFreeSpace());
+    }
+
+    /**
+     * Returns the number of available bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of available bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getUsableSpace(File file)
+    {
+        return handleLargeFileSystem(file.getUsableSpace());
+    }
+
+    /**
+     * Returns the {@link FileStore} representing the file store where a file
+     * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+     * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+     * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+     * for more information.
+     *
+     * @param path the path to the file
+     * @return the file store where the file is stored
+     */
+    public static FileStore getFileStore(Path path) throws IOException
+    {
+        return new SafeFileStore(Files.getFileStore(path));
+    }
+
+    /**
+     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+     * @param size returned by the Java's FileStore methods
+     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     */
+    private static long handleLargeFileSystem(long size)
+    {
+        return size < 0 ? Long.MAX_VALUE : size;
+    }
+
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private FileUtils()
+    {
+    }
+
+    /**
+     * FileStore decorator used to safely handle large file system.
+     *
+     * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+     * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+     * return {@code Long.MAX_VALUE} if the size overflow.</p>
+     *
+     * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+     */
+    private static final class SafeFileStore extends FileStore
+    {
+        /**
+         * The decorated {@code FileStore}
+         */
+        private final FileStore fileStore;
+
+        public SafeFileStore(FileStore fileStore)
+        {
+            this.fileStore = fileStore;
+        }
+
+        @Override
+        public String name()
+        {
+            return fileStore.name();
+        }
+
+        @Override
+        public String type()
+        {
+            return fileStore.type();
+        }
+
+        @Override
+        public boolean isReadOnly()
+        {
+            return fileStore.isReadOnly();
+        }
+
+        @Override
+        public long getTotalSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getTotalSpace());
+        }
+
+        @Override
+        public long getUsableSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUsableSpace());
+        }
+
+        @Override
+        public long getUnallocatedSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+        {
+            return fileStore.supportsFileAttributeView(type);
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(String name)
+        {
+            return fileStore.supportsFileAttributeView(name);
+        }
+
+        @Override
+        public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+        {
+            return fileStore.getFileStoreAttributeView(type);
+        }
+
+        @Override
+        public Object getAttribute(String attribute) throws IOException
+        {
+            return fileStore.getAttribute(attribute);
+        }
+    }
 }


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


[04/10] cassandra git commit: Prevent integer overflow on exabyte filesystems

Posted by bl...@apache.org.
Prevent integer overflow on exabyte filesystems

patch by Matt Wringe and Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13067


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

Branch: refs/heads/trunk
Commit: 270f690ff6047cc3e797a3f34b7efa26e7232183
Parents: 739cd2b
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:51:03 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 16:51:03 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 182 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 36c34a1..f712333 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.11
+ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
  * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
  * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
  * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 981026d..90a82fe 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.IOException;
 import java.net.*;
 import java.nio.file.FileStore;
-import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -56,6 +55,9 @@ import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.memory.*;
 
+import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
+import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+
 public class DatabaseDescriptor
 {
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -530,7 +532,7 @@ public class DatabaseDescriptor
             try
             {
                 // use 1/4 of available space.  See discussion on #10013 and #10199
-                minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
+                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
             }
             catch (IOException e)
             {
@@ -576,7 +578,7 @@ public class DatabaseDescriptor
 
             try
             {
-                dataFreeBytes += guessFileStore(datadir).getUnallocatedSpace();
+                dataFreeBytes = saturatedSum(dataFreeBytes, guessFileStore(datadir).getUnallocatedSpace());
             }
             catch (IOException e)
             {
@@ -585,9 +587,9 @@ public class DatabaseDescriptor
                                                                datadir), e);
             }
         }
-        if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
+        if (dataFreeBytes < 64 * ONE_GB)
             logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
-                        dataFreeBytes / 1048576);
+                        dataFreeBytes / ONE_MB);
 
 
         if (conf.commitlog_directory.equals(conf.saved_caches_directory))
@@ -697,6 +699,20 @@ public class DatabaseDescriptor
             throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
     }
 
+    /**
+     * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+     *
+     * @param left the left operand
+     * @param right the right operand
+     * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+     */
+    private static long saturatedSum(long left, long right)
+    {
+        assert left >= 0 && right >= 0;
+        long sum = left + right;
+        return sum < 0 ? Long.MAX_VALUE : sum;
+    }
+
     private static FileStore guessFileStore(String dir) throws IOException
     {
         Path path = Paths.get(dir);
@@ -704,7 +720,7 @@ public class DatabaseDescriptor
         {
             try
             {
-                return Files.getFileStore(path);
+                return FileUtils.getFileStore(path);
             }
             catch (IOException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 2b3662f..fa76b61 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -482,7 +482,7 @@ public class Directories
 
         public long getAvailableSpace()
         {
-            return location.getUsableSpace();
+            return FileUtils.getUsableSpace(location);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 8d122dd..bf0fae5 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -21,11 +21,12 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.*;
+import java.nio.file.attribute.FileAttributeView;
+import java.nio.file.attribute.FileStoreAttributeView;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.cassandra.config.Config;
 import sun.nio.ch.DirectBuffer;
 
 import org.slf4j.Logger;
@@ -45,10 +46,10 @@ import static org.apache.cassandra.utils.Throwables.merge;
 public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-    private static final double KB = 1024d;
-    private static final double MB = 1024*1024d;
-    private static final double GB = 1024*1024*1024d;
-    private static final double TB = 1024*1024*1024*1024d;
+    public static final long ONE_KB = 1024;
+    public static final long ONE_MB = 1024 * ONE_KB;
+    public static final long ONE_GB = 1024 * ONE_MB;
+    public static final long ONE_TB = 1024 * ONE_GB;
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
@@ -330,27 +331,27 @@ public final class FileUtils
     public static String stringifyFileSize(double value)
     {
         double d;
-        if ( value >= TB )
+        if ( value >= ONE_TB )
         {
-            d = value / TB;
+            d = value / ONE_TB;
             String val = df.format(d);
             return val + " TB";
         }
-        else if ( value >= GB )
+        else if ( value >= ONE_GB )
         {
-            d = value / GB;
+            d = value / ONE_GB;
             String val = df.format(d);
             return val + " GB";
         }
-        else if ( value >= MB )
+        else if ( value >= ONE_MB )
         {
-            d = value / MB;
+            d = value / ONE_MB;
             String val = df.format(d);
             return val + " MB";
         }
-        else if ( value >= KB )
+        else if ( value >= ONE_KB )
         {
-            d = value / KB;
+            d = value / ONE_KB;
             String val = df.format(d);
             return val + " KB";
         }
@@ -478,4 +479,159 @@ public final class FileUtils
     {
         fsErrorHandler.getAndSet(handler);
     }
+
+    /**
+     * Returns the size of the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+     * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+     *
+     * @param file the partition
+     * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+     */
+    public static long getTotalSpace(File file)
+    {
+        return handleLargeFileSystem(file.getTotalSpace());
+    }
+
+    /**
+     * Returns the number of unallocated bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of unallocated bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getFreeSpace(File file)
+    {
+        return handleLargeFileSystem(file.getFreeSpace());
+    }
+
+    /**
+     * Returns the number of available bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of available bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getUsableSpace(File file)
+    {
+        return handleLargeFileSystem(file.getUsableSpace());
+    }
+
+    /**
+     * Returns the {@link FileStore} representing the file store where a file
+     * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+     * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+     * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+     * for more information.
+     *
+     * @param path the path to the file
+     * @return the file store where the file is stored
+     */
+    public static FileStore getFileStore(Path path) throws IOException
+    {
+        return new SafeFileStore(Files.getFileStore(path));
+    }
+
+    /**
+     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+     * @param size returned by the Java's FileStore methods
+     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     */
+    private static long handleLargeFileSystem(long size)
+    {
+        return size < 0 ? Long.MAX_VALUE : size;
+    }
+
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private FileUtils()
+    {
+    }
+
+    /**
+     * FileStore decorator used to safely handle large file system.
+     *
+     * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+     * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+     * return {@code Long.MAX_VALUE} if the size overflow.</p>
+     *
+     * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+     */
+    private static final class SafeFileStore extends FileStore
+    {
+        /**
+         * The decorated {@code FileStore}
+         */
+        private final FileStore fileStore;
+
+        public SafeFileStore(FileStore fileStore)
+        {
+            this.fileStore = fileStore;
+        }
+
+        @Override
+        public String name()
+        {
+            return fileStore.name();
+        }
+
+        @Override
+        public String type()
+        {
+            return fileStore.type();
+        }
+
+        @Override
+        public boolean isReadOnly()
+        {
+            return fileStore.isReadOnly();
+        }
+
+        @Override
+        public long getTotalSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getTotalSpace());
+        }
+
+        @Override
+        public long getUsableSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUsableSpace());
+        }
+
+        @Override
+        public long getUnallocatedSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+        {
+            return fileStore.supportsFileAttributeView(type);
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(String name)
+        {
+            return fileStore.supportsFileAttributeView(name);
+        }
+
+        @Override
+        public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+        {
+            return fileStore.getFileStoreAttributeView(type);
+        }
+
+        @Override
+        public Object getAttribute(String attribute) throws IOException
+        {
+            return fileStore.getAttribute(attribute);
+        }
+    }
 }


---------------------------------------------------------------------
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/1a70dede
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1a70dede
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1a70dede

Branch: refs/heads/trunk
Commit: 1a70dede3a33a2e8bf107f44139580b4ace1c9ff
Parents: 3960260 270f690
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:55:26 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:08:19 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 1525289,f712333..1f42c70
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,22 -1,5 +1,23 @@@
 -2.2.11
 - * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
 +3.0.15
 + * 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)
 + Merged from 2.2:
++ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067)
   * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
   * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
   * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Directories.java
index 68aa6be,fa76b61..ae7700c
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@@ -523,26 -482,7 +523,25 @@@ public class Directorie
  
          public long getAvailableSpace()
          {
-             long availableSpace = location.getUsableSpace() - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 -            return FileUtils.getUsableSpace(location);
++            long availableSpace = FileUtils.getUsableSpace(location) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 +            return availableSpace > 0 ? availableSpace : 0;
 +        }
 +
 +        @Override
 +        public boolean equals(Object o)
 +        {
 +            if (this == o) return true;
 +            if (o == null || getClass() != o.getClass()) return false;
 +
 +            DataDirectory that = (DataDirectory) o;
 +
 +            return location.equals(that.location);
- 
 +        }
 +
 +        @Override
 +        public int hashCode()
 +        {
 +            return location.hashCode();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index 0bfbbb1,bf0fae5..80df67b
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -20,17 -20,14 +20,19 @@@ package org.apache.cassandra.io.util
  import java.io.*;
  import java.nio.ByteBuffer;
  import java.nio.channels.FileChannel;
 +import java.nio.charset.Charset;
 +import java.nio.charset.StandardCharsets;
  import java.nio.file.*;
+ import java.nio.file.attribute.FileAttributeView;
+ import java.nio.file.attribute.FileStoreAttributeView;
  import java.text.DecimalFormat;
  import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
  import java.util.concurrent.atomic.AtomicReference;
 -
 -import sun.nio.ch.DirectBuffer;
 +import java.util.function.Consumer;
 +import java.util.function.Predicate;
 +import java.util.stream.StreamSupport;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -49,13 -45,11 +51,13 @@@ import static org.apache.cassandra.util
  
  public final class FileUtils
  {
 +    public static final Charset CHARSET = StandardCharsets.UTF_8;
 +
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-     private static final double KB = 1024d;
-     private static final double MB = 1024*1024d;
-     private static final double GB = 1024*1024*1024d;
-     private static final double TB = 1024*1024*1024*1024d;
+     public static final long ONE_KB = 1024;
+     public static final long ONE_MB = 1024 * ONE_KB;
+     public static final long ONE_GB = 1024 * ONE_MB;
+     public static final long ONE_TB = 1024 * ONE_GB;
  
      private static final DecimalFormat df = new DecimalFormat("#.##");
      private static final boolean canCleanDirectBuffers;


---------------------------------------------------------------------
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/1a70dede
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1a70dede
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1a70dede

Branch: refs/heads/cassandra-3.0
Commit: 1a70dede3a33a2e8bf107f44139580b4ace1c9ff
Parents: 3960260 270f690
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:55:26 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:08:19 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 1525289,f712333..1f42c70
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,22 -1,5 +1,23 @@@
 -2.2.11
 - * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
 +3.0.15
 + * 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)
 + Merged from 2.2:
++ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067)
   * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
   * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
   * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Directories.java
index 68aa6be,fa76b61..ae7700c
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@@ -523,26 -482,7 +523,25 @@@ public class Directorie
  
          public long getAvailableSpace()
          {
-             long availableSpace = location.getUsableSpace() - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 -            return FileUtils.getUsableSpace(location);
++            long availableSpace = FileUtils.getUsableSpace(location) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 +            return availableSpace > 0 ? availableSpace : 0;
 +        }
 +
 +        @Override
 +        public boolean equals(Object o)
 +        {
 +            if (this == o) return true;
 +            if (o == null || getClass() != o.getClass()) return false;
 +
 +            DataDirectory that = (DataDirectory) o;
 +
 +            return location.equals(that.location);
- 
 +        }
 +
 +        @Override
 +        public int hashCode()
 +        {
 +            return location.hashCode();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index 0bfbbb1,bf0fae5..80df67b
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -20,17 -20,14 +20,19 @@@ package org.apache.cassandra.io.util
  import java.io.*;
  import java.nio.ByteBuffer;
  import java.nio.channels.FileChannel;
 +import java.nio.charset.Charset;
 +import java.nio.charset.StandardCharsets;
  import java.nio.file.*;
+ import java.nio.file.attribute.FileAttributeView;
+ import java.nio.file.attribute.FileStoreAttributeView;
  import java.text.DecimalFormat;
  import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
  import java.util.concurrent.atomic.AtomicReference;
 -
 -import sun.nio.ch.DirectBuffer;
 +import java.util.function.Consumer;
 +import java.util.function.Predicate;
 +import java.util.stream.StreamSupport;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -49,13 -45,11 +51,13 @@@ import static org.apache.cassandra.util
  
  public final class FileUtils
  {
 +    public static final Charset CHARSET = StandardCharsets.UTF_8;
 +
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-     private static final double KB = 1024d;
-     private static final double MB = 1024*1024d;
-     private static final double GB = 1024*1024*1024d;
-     private static final double TB = 1024*1024*1024*1024d;
+     public static final long ONE_KB = 1024;
+     public static final long ONE_MB = 1024 * ONE_KB;
+     public static final long ONE_GB = 1024 * ONE_MB;
+     public static final long ONE_TB = 1024 * ONE_GB;
  
      private static final DecimalFormat df = new DecimalFormat("#.##");
      private static final boolean canCleanDirectBuffers;


---------------------------------------------------------------------
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/303dba65
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/303dba65
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/303dba65

Branch: refs/heads/cassandra-3.11
Commit: 303dba6504ba069b5ea92ee18a47d3ba87c1563e
Parents: 47a2839 1a70ded
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 17:09:06 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:11:24 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  26 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 191 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index b778df6,1f42c70..145a746
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -17,9 -13,11 +17,10 @@@ Merged from 3.0
   * 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)
 - Merged from 2.2:
 +Merged from 2.2:
+  * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067)
   * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
   * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
   * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 87b388e,aba7617..ad43565
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -19,10 -19,8 +19,9 @@@ package org.apache.cassandra.config
  
  import java.io.File;
  import java.io.IOException;
 +import java.lang.reflect.Constructor;
  import java.net.*;
  import java.nio.file.FileStore;
- import java.nio.file.Files;
  import java.nio.file.NoSuchFileException;
  import java.nio.file.Path;
  import java.nio.file.Paths;
@@@ -46,25 -40,24 +45,27 @@@ import org.apache.cassandra.config.Conf
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.io.FSWriteError;
 -import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.util.DiskOptimizationStrategy;
  import org.apache.cassandra.io.util.FileUtils;
 -import org.apache.cassandra.locator.*;
 -import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy;
 +import org.apache.cassandra.io.util.SsdDiskOptimizationStrategy;
 +import org.apache.cassandra.locator.DynamicEndpointSnitch;
 +import org.apache.cassandra.locator.EndpointSnitchInfo;
 +import org.apache.cassandra.locator.IEndpointSnitch;
 +import org.apache.cassandra.locator.SeedProvider;
 +import org.apache.cassandra.net.BackPressureStrategy;
 +import org.apache.cassandra.net.RateBasedBackPressure;
  import org.apache.cassandra.scheduler.IRequestScheduler;
  import org.apache.cassandra.scheduler.NoScheduler;
 -import org.apache.cassandra.service.CacheService;
 -import org.apache.cassandra.thrift.ThriftServer;
 +import org.apache.cassandra.security.EncryptionContext;
 +import org.apache.cassandra.service.CacheService.CacheType;
 +import org.apache.cassandra.thrift.ThriftServer.ThriftServerType;
  import org.apache.cassandra.utils.FBUtilities;
 -import org.apache.cassandra.utils.memory.*;
 +
 +import org.apache.commons.lang3.StringUtils;
  
+ import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
 -import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+ 
  public class DatabaseDescriptor
  {
      private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@@ -429,53 -443,96 +430,53 @@@
          if (conf.native_transport_max_frame_size_in_mb <= 0)
              throw new ConfigurationException("native_transport_max_frame_size_in_mb must be positive, but was " + conf.native_transport_max_frame_size_in_mb, false);
  
 -        // fail early instead of OOMing (see CASSANDRA-8116)
 -        if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads == Integer.MAX_VALUE)
 -            throw new ConfigurationException("The hsha rpc_server_type is not compatible with an rpc_max_threads " +
 -                                             "setting of 'unlimited'.  Please see the comments in cassandra.yaml " +
 -                                             "for rpc_server_type and rpc_max_threads.",
 -                                             false);
 -        if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads > (FBUtilities.getAvailableProcessors() * 2 + 1024))
 -            logger.warn("rpc_max_threads setting of {} may be too high for the hsha server and cause unnecessary thread contention, reducing performance", conf.rpc_max_threads);
 +        // if data dirs, commitlog dir, or saved caches dir are set in cassandra.yaml, use that.  Otherwise,
 +        // use -Dcassandra.storagedir (set in cassandra-env.sh) as the parent dir for data/, commitlog/, and saved_caches/
 +        if (conf.commitlog_directory == null)
 +        {
 +            conf.commitlog_directory = storagedirFor("commitlog");
 +        }
  
 -        /* end point snitch */
 -        if (conf.endpoint_snitch == null)
 +        if (conf.hints_directory == null)
          {
 -            throw new ConfigurationException("Missing endpoint_snitch directive", false);
 +            conf.hints_directory = storagedirFor("hints");
          }
 -        snitch = createEndpointSnitch(conf.endpoint_snitch);
 -        EndpointSnitchInfo.create();
  
 -        localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
 -        localComparator = new Comparator<InetAddress>()
 +        if (conf.cdc_raw_directory == null)
          {
 -            public int compare(InetAddress endpoint1, InetAddress endpoint2)
 -            {
 -                boolean local1 = localDC.equals(snitch.getDatacenter(endpoint1));
 -                boolean local2 = localDC.equals(snitch.getDatacenter(endpoint2));
 -                if (local1 && !local2)
 -                    return -1;
 -                if (local2 && !local1)
 -                    return 1;
 -                return 0;
 -            }
 -        };
 +            conf.cdc_raw_directory = storagedirFor("cdc_raw");
 +        }
  
 -        /* Request Scheduler setup */
 -        requestSchedulerOptions = conf.request_scheduler_options;
 -        if (conf.request_scheduler != null)
 +        if (conf.commitlog_total_space_in_mb == null)
          {
 +            int preferredSize = 8192;
 +            int minSize = 0;
              try
              {
 -                if (requestSchedulerOptions == null)
 -                {
 -                    requestSchedulerOptions = new RequestSchedulerOptions();
 -                }
 -                Class<?> cls = Class.forName(conf.request_scheduler);
 -                requestScheduler = (IRequestScheduler) cls.getConstructor(RequestSchedulerOptions.class).newInstance(requestSchedulerOptions);
 +                // use 1/4 of available space.  See discussion on #10013 and #10199
-                 minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
++                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
              }
 -            catch (ClassNotFoundException e)
 +            catch (IOException e)
              {
 -                throw new ConfigurationException("Invalid Request Scheduler class " + conf.request_scheduler, false);
 +                logger.debug("Error checking disk space", e);
 +                throw new ConfigurationException(String.format("Unable to check disk space available to %s. Perhaps the Cassandra user does not have the necessary permissions",
 +                                                               conf.commitlog_directory), e);
              }
 -            catch (Exception e)
 +            if (minSize < preferredSize)
              {
 -                throw new ConfigurationException("Unable to instantiate request scheduler", e);
 +                logger.warn("Small commitlog volume detected at {}; setting commitlog_total_space_in_mb to {}.  You can override this in cassandra.yaml",
 +                            conf.commitlog_directory, minSize);
 +                conf.commitlog_total_space_in_mb = minSize;
 +            }
 +            else
 +            {
 +                conf.commitlog_total_space_in_mb = preferredSize;
              }
 -        }
 -        else
 -        {
 -            requestScheduler = new NoScheduler();
          }
  
 -        if (conf.request_scheduler_id == RequestSchedulerId.keyspace)
 -        {
 -            requestSchedulerId = conf.request_scheduler_id;
 -        }
 -        else
 -        {
 -            // Default to Keyspace
 -            requestSchedulerId = RequestSchedulerId.keyspace;
 -        }
 -
 -        // if data dirs, commitlog dir, or saved caches dir are set in cassandra.yaml, use that.  Otherwise,
 -        // use -Dcassandra.storagedir (set in cassandra-env.sh) as the parent dir for data/, commitlog/, and saved_caches/
 -        if (conf.commitlog_directory == null)
 -        {
 -            conf.commitlog_directory = System.getProperty("cassandra.storagedir", null);
 -            if (conf.commitlog_directory == null)
 -                throw new ConfigurationException("commitlog_directory is missing and -Dcassandra.storagedir is not set", false);
 -            conf.commitlog_directory += File.separator + "commitlog";
 -        }
 -
 -        if (conf.hints_directory == null)
 +        if (conf.cdc_total_space_in_mb == 0)
          {
 -            conf.hints_directory = System.getProperty("cassandra.storagedir", null);
 -            if (conf.hints_directory == null)
 -                throw new ConfigurationException("hints_directory is missing and -Dcassandra.storagedir is not set", false);
 -            conf.hints_directory += File.separator + "hints";
 -        }
 -
 -        if (conf.commitlog_total_space_in_mb == null)
 -        {
 -            int preferredSize = 8192;
 +            int preferredSize = 4096;
              int minSize = 0;
              try
              {
@@@ -536,11 -594,11 +537,10 @@@
                                                                 datadir), e);
              }
          }
-         if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
 -        if (dataFreeBytes < 64 * ONE_GB)
 -            logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
 -                        dataFreeBytes / ONE_MB);
 -
++        if (dataFreeBytes < 64 * ONE_GB) // 64 GB
 +            logger.warn("Only {} free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
 +                        FBUtilities.prettyPrintMemory(dataFreeBytes));
  
- 
          if (conf.commitlog_directory.equals(conf.saved_caches_directory))
              throw new ConfigurationException("saved_caches_directory must not be the same as the commitlog_directory", false);
          if (conf.commitlog_directory.equals(conf.hints_directory))
@@@ -970,16 -699,62 +970,30 @@@
          }
          try
          {
 -            Class<?> seedProviderClass = Class.forName(conf.seed_provider.class_name);
 -            seedProvider = (SeedProvider)seedProviderClass.getConstructor(Map.class).newInstance(conf.seed_provider.parameters);
 +            partitioner = FBUtilities.newPartitioner(System.getProperty(Config.PROPERTY_PREFIX + "partitioner", conf.partitioner));
          }
 -        // there are about 5 checked exceptions that could be thrown here.
          catch (Exception e)
          {
 -            throw new ConfigurationException(e.getMessage() + "\nFatal configuration error; unable to start server.  See log for stacktrace.", true);
 -        }
 -        if (seedProvider.getSeeds().size() == 0)
 -            throw new ConfigurationException("The seed provider lists no seeds.", false);
 -
 -        if (conf.user_defined_function_fail_timeout < 0)
 -            throw new ConfigurationException("user_defined_function_fail_timeout must not be negative", false);
 -        if (conf.user_defined_function_warn_timeout < 0)
 -            throw new ConfigurationException("user_defined_function_warn_timeout must not be negative", false);
 -
 -        if (conf.user_defined_function_fail_timeout < conf.user_defined_function_warn_timeout)
 -            throw new ConfigurationException("user_defined_function_warn_timeout must less than user_defined_function_fail_timeout", false);
 -
 -        if (conf.max_mutation_size_in_kb == null)
 -            conf.max_mutation_size_in_kb = conf.commitlog_segment_size_in_mb * 1024 / 2;
 -        else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_in_kb)
 -            throw new ConfigurationException("commitlog_segment_size_in_mb must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 -
 -        // native transport encryption options
 -        if (conf.native_transport_port_ssl != null
 -            && conf.native_transport_port_ssl.intValue() != conf.native_transport_port.intValue()
 -            && !conf.client_encryption_options.enabled)
 -        {
 -            throw new ConfigurationException("Encryption must be enabled in client_encryption_options for native_transport_port_ssl", false);
 +            throw new ConfigurationException("Invalid partitioner class " + conf.partitioner, false);
          }
  
 -        if (conf.max_value_size_in_mb == null || conf.max_value_size_in_mb <= 0)
 -            throw new ConfigurationException("max_value_size_in_mb must be positive", false);
 -
 -        if (conf.otc_coalescing_enough_coalesced_messages > 128)
 -            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be smaller than 128", false);
 -
 -        if (conf.otc_coalescing_enough_coalesced_messages <= 0)
 -            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
 +        paritionerName = partitioner.getClass().getCanonicalName();
      }
  
+     /**
+      * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+      *
+      * @param left the left operand
+      * @param right the right operand
+      * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+      */
+     private static long saturatedSum(long left, long right)
+     {
+         assert left >= 0 && right >= 0;
+         long sum = left + right;
+         return sum < 0 ? Long.MAX_VALUE : sum;
+     }
+ 
      private static FileStore guessFileStore(String dir) throws IOException
      {
          Path path = Paths.get(dir);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/303dba65/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index c6e4e63,80df67b..24017cf
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -23,7 -23,8 +23,9 @@@ import java.nio.channels.FileChannel
  import java.nio.charset.Charset;
  import java.nio.charset.StandardCharsets;
  import java.nio.file.*;
 +import java.nio.file.attribute.BasicFileAttributes;
+ import java.nio.file.attribute.FileAttributeView;
+ import java.nio.file.attribute.FileStoreAttributeView;
  import java.text.DecimalFormat;
  import java.util.Arrays;
  import java.util.Collections;
@@@ -54,14 -54,14 +56,14 @@@ public final class FileUtil
      public static final Charset CHARSET = StandardCharsets.UTF_8;
  
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-     private static final double KB = 1024d;
-     private static final double MB = 1024*1024d;
-     private static final double GB = 1024*1024*1024d;
-     private static final double TB = 1024*1024*1024*1024d;
+     public static final long ONE_KB = 1024;
+     public static final long ONE_MB = 1024 * ONE_KB;
+     public static final long ONE_GB = 1024 * ONE_MB;
+     public static final long ONE_TB = 1024 * ONE_GB;
  
      private static final DecimalFormat df = new DecimalFormat("#.##");
 -    private static final boolean canCleanDirectBuffers;
 -    private static final AtomicReference<FSErrorHandler> fsErrorHandler = new AtomicReference<>();
 +    public static final boolean isCleanerAvailable;
 +    private static final AtomicReference<Optional<FSErrorHandler>> fsErrorHandler = new AtomicReference<>(Optional.empty());
  
      static
      {
@@@ -415,29 -418,29 +417,29 @@@
      public static String stringifyFileSize(double value)
      {
          double d;
-         if ( value >= TB )
+         if ( value >= ONE_TB )
          {
-             d = value / TB;
+             d = value / ONE_TB;
              String val = df.format(d);
 -            return val + " TB";
 +            return val + " TiB";
          }
-         else if ( value >= GB )
+         else if ( value >= ONE_GB )
          {
-             d = value / GB;
+             d = value / ONE_GB;
              String val = df.format(d);
 -            return val + " GB";
 +            return val + " GiB";
          }
-         else if ( value >= MB )
+         else if ( value >= ONE_MB )
          {
-             d = value / MB;
+             d = value / ONE_MB;
              String val = df.format(d);
 -            return val + " MB";
 +            return val + " MiB";
          }
-         else if ( value >= KB )
+         else if ( value >= ONE_KB )
          {
-             d = value / KB;
+             d = value / ONE_KB;
              String val = df.format(d);
 -            return val + " KB";
 +            return val + " KiB";
          }
          else
          {
@@@ -606,6 -603,161 +608,161 @@@
  
      public static void setFSErrorHandler(FSErrorHandler handler)
      {
 -        fsErrorHandler.getAndSet(handler);
 +        fsErrorHandler.getAndSet(Optional.ofNullable(handler));
      }
+ 
+     /**
+      * Returns the size of the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+      * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+      *
+      * @param file the partition
+      * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+      */
+     public static long getTotalSpace(File file)
+     {
+         return handleLargeFileSystem(file.getTotalSpace());
+     }
+ 
+     /**
+      * Returns the number of unallocated bytes on the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+      * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+      *
+      * @param file the partition
+      * @return the number of unallocated bytes on the partition or {@code 0L}
+      * if the abstract pathname does not name a partition.
+      */
+     public static long getFreeSpace(File file)
+     {
+         return handleLargeFileSystem(file.getFreeSpace());
+     }
+ 
+     /**
+      * Returns the number of available bytes on the specified partition.
+      * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+      * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+      *
+      * @param file the partition
+      * @return the number of available bytes on the partition or {@code 0L}
+      * if the abstract pathname does not name a partition.
+      */
+     public static long getUsableSpace(File file)
+     {
+         return handleLargeFileSystem(file.getUsableSpace());
+     }
+ 
+     /**
+      * Returns the {@link FileStore} representing the file store where a file
+      * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+      * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+      * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+      * for more information.
+      *
+      * @param path the path to the file
+      * @return the file store where the file is stored
+      */
+     public static FileStore getFileStore(Path path) throws IOException
+     {
+         return new SafeFileStore(Files.getFileStore(path));
+     }
+ 
+     /**
+      * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+      * @param size returned by the Java's FileStore methods
+      * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+      */
+     private static long handleLargeFileSystem(long size)
+     {
+         return size < 0 ? Long.MAX_VALUE : size;
+     }
+ 
+     /**
+      * Private constructor as the class contains only static methods.
+      */
+     private FileUtils()
+     {
+     }
+ 
+     /**
+      * FileStore decorator used to safely handle large file system.
+      *
+      * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+      * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+      * return {@code Long.MAX_VALUE} if the size overflow.</p>
+      *
+      * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+      */
+     private static final class SafeFileStore extends FileStore
+     {
+         /**
+          * The decorated {@code FileStore}
+          */
+         private final FileStore fileStore;
+ 
+         public SafeFileStore(FileStore fileStore)
+         {
+             this.fileStore = fileStore;
+         }
+ 
+         @Override
+         public String name()
+         {
+             return fileStore.name();
+         }
+ 
+         @Override
+         public String type()
+         {
+             return fileStore.type();
+         }
+ 
+         @Override
+         public boolean isReadOnly()
+         {
+             return fileStore.isReadOnly();
+         }
+ 
+         @Override
+         public long getTotalSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getTotalSpace());
+         }
+ 
+         @Override
+         public long getUsableSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getUsableSpace());
+         }
+ 
+         @Override
+         public long getUnallocatedSpace() throws IOException
+         {
+             return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+         }
+ 
+         @Override
+         public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+         {
+             return fileStore.supportsFileAttributeView(type);
+         }
+ 
+         @Override
+         public boolean supportsFileAttributeView(String name)
+         {
+             return fileStore.supportsFileAttributeView(name);
+         }
+ 
+         @Override
+         public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+         {
+             return fileStore.getFileStoreAttributeView(type);
+         }
+ 
+         @Override
+         public Object getAttribute(String attribute) throws IOException
+         {
+             return fileStore.getAttribute(attribute);
+         }
+     }
  }


---------------------------------------------------------------------
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/1a70dede
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1a70dede
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1a70dede

Branch: refs/heads/cassandra-3.11
Commit: 1a70dede3a33a2e8bf107f44139580b4ace1c9ff
Parents: 3960260 270f690
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:55:26 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:08:19 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 1525289,f712333..1f42c70
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,22 -1,5 +1,23 @@@
 -2.2.11
 - * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
 +3.0.15
 + * 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)
 + Merged from 2.2:
++ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067)
   * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
   * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
   * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Directories.java
index 68aa6be,fa76b61..ae7700c
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@@ -523,26 -482,7 +523,25 @@@ public class Directorie
  
          public long getAvailableSpace()
          {
-             long availableSpace = location.getUsableSpace() - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 -            return FileUtils.getUsableSpace(location);
++            long availableSpace = FileUtils.getUsableSpace(location) - DatabaseDescriptor.getMinFreeSpacePerDriveInBytes();
 +            return availableSpace > 0 ? availableSpace : 0;
 +        }
 +
 +        @Override
 +        public boolean equals(Object o)
 +        {
 +            if (this == o) return true;
 +            if (o == null || getClass() != o.getClass()) return false;
 +
 +            DataDirectory that = (DataDirectory) o;
 +
 +            return location.equals(that.location);
- 
 +        }
 +
 +        @Override
 +        public int hashCode()
 +        {
 +            return location.hashCode();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1a70dede/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index 0bfbbb1,bf0fae5..80df67b
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -20,17 -20,14 +20,19 @@@ package org.apache.cassandra.io.util
  import java.io.*;
  import java.nio.ByteBuffer;
  import java.nio.channels.FileChannel;
 +import java.nio.charset.Charset;
 +import java.nio.charset.StandardCharsets;
  import java.nio.file.*;
+ import java.nio.file.attribute.FileAttributeView;
+ import java.nio.file.attribute.FileStoreAttributeView;
  import java.text.DecimalFormat;
  import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
  import java.util.concurrent.atomic.AtomicReference;
 -
 -import sun.nio.ch.DirectBuffer;
 +import java.util.function.Consumer;
 +import java.util.function.Predicate;
 +import java.util.stream.StreamSupport;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -49,13 -45,11 +51,13 @@@ import static org.apache.cassandra.util
  
  public final class FileUtils
  {
 +    public static final Charset CHARSET = StandardCharsets.UTF_8;
 +
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-     private static final double KB = 1024d;
-     private static final double MB = 1024*1024d;
-     private static final double GB = 1024*1024*1024d;
-     private static final double TB = 1024*1024*1024*1024d;
+     public static final long ONE_KB = 1024;
+     public static final long ONE_MB = 1024 * ONE_KB;
+     public static final long ONE_GB = 1024 * ONE_MB;
+     public static final long ONE_TB = 1024 * ONE_GB;
  
      private static final DecimalFormat df = new DecimalFormat("#.##");
      private static final boolean canCleanDirectBuffers;


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


[02/10] cassandra git commit: Prevent integer overflow on exabyte filesystems

Posted by bl...@apache.org.
Prevent integer overflow on exabyte filesystems

patch by Matt Wringe and Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13067


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

Branch: refs/heads/cassandra-3.0
Commit: 270f690ff6047cc3e797a3f34b7efa26e7232183
Parents: 739cd2b
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 16:51:03 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 16:51:03 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  28 ++-
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 182 +++++++++++++++++--
 4 files changed, 193 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 36c34a1..f712333 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.11
+ * Prevent integer overflow on exabyte filesystems (CASSANDRA-13067) 
  * Fix queries with LIMIT and filtering on clustering columns (CASSANDRA-11223)
  * Fix potential NPE when resume bootstrap fails (CASSANDRA-13272)
  * Fix toJSONString for the UDT, tuple and collection types (CASSANDRA-13592)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 981026d..90a82fe 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.IOException;
 import java.net.*;
 import java.nio.file.FileStore;
-import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -56,6 +55,9 @@ import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.memory.*;
 
+import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
+import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
+
 public class DatabaseDescriptor
 {
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
@@ -530,7 +532,7 @@ public class DatabaseDescriptor
             try
             {
                 // use 1/4 of available space.  See discussion on #10013 and #10199
-                minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
+                minSize = Ints.saturatedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
             }
             catch (IOException e)
             {
@@ -576,7 +578,7 @@ public class DatabaseDescriptor
 
             try
             {
-                dataFreeBytes += guessFileStore(datadir).getUnallocatedSpace();
+                dataFreeBytes = saturatedSum(dataFreeBytes, guessFileStore(datadir).getUnallocatedSpace());
             }
             catch (IOException e)
             {
@@ -585,9 +587,9 @@ public class DatabaseDescriptor
                                                                datadir), e);
             }
         }
-        if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
+        if (dataFreeBytes < 64 * ONE_GB)
             logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
-                        dataFreeBytes / 1048576);
+                        dataFreeBytes / ONE_MB);
 
 
         if (conf.commitlog_directory.equals(conf.saved_caches_directory))
@@ -697,6 +699,20 @@ public class DatabaseDescriptor
             throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
     }
 
+    /**
+     * Computes the sum of the 2 specified positive values returning {@code Long.MAX_VALUE} if the sum overflow.
+     *
+     * @param left the left operand
+     * @param right the right operand
+     * @return the sum of the 2 specified positive values of {@code Long.MAX_VALUE} if the sum overflow.
+     */
+    private static long saturatedSum(long left, long right)
+    {
+        assert left >= 0 && right >= 0;
+        long sum = left + right;
+        return sum < 0 ? Long.MAX_VALUE : sum;
+    }
+
     private static FileStore guessFileStore(String dir) throws IOException
     {
         Path path = Paths.get(dir);
@@ -704,7 +720,7 @@ public class DatabaseDescriptor
         {
             try
             {
-                return Files.getFileStore(path);
+                return FileUtils.getFileStore(path);
             }
             catch (IOException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 2b3662f..fa76b61 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -482,7 +482,7 @@ public class Directories
 
         public long getAvailableSpace()
         {
-            return location.getUsableSpace();
+            return FileUtils.getUsableSpace(location);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/270f690f/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 8d122dd..bf0fae5 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -21,11 +21,12 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.*;
+import java.nio.file.attribute.FileAttributeView;
+import java.nio.file.attribute.FileStoreAttributeView;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.cassandra.config.Config;
 import sun.nio.ch.DirectBuffer;
 
 import org.slf4j.Logger;
@@ -45,10 +46,10 @@ import static org.apache.cassandra.utils.Throwables.merge;
 public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
-    private static final double KB = 1024d;
-    private static final double MB = 1024*1024d;
-    private static final double GB = 1024*1024*1024d;
-    private static final double TB = 1024*1024*1024*1024d;
+    public static final long ONE_KB = 1024;
+    public static final long ONE_MB = 1024 * ONE_KB;
+    public static final long ONE_GB = 1024 * ONE_MB;
+    public static final long ONE_TB = 1024 * ONE_GB;
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
@@ -330,27 +331,27 @@ public final class FileUtils
     public static String stringifyFileSize(double value)
     {
         double d;
-        if ( value >= TB )
+        if ( value >= ONE_TB )
         {
-            d = value / TB;
+            d = value / ONE_TB;
             String val = df.format(d);
             return val + " TB";
         }
-        else if ( value >= GB )
+        else if ( value >= ONE_GB )
         {
-            d = value / GB;
+            d = value / ONE_GB;
             String val = df.format(d);
             return val + " GB";
         }
-        else if ( value >= MB )
+        else if ( value >= ONE_MB )
         {
-            d = value / MB;
+            d = value / ONE_MB;
             String val = df.format(d);
             return val + " MB";
         }
-        else if ( value >= KB )
+        else if ( value >= ONE_KB )
         {
-            d = value / KB;
+            d = value / ONE_KB;
             String val = df.format(d);
             return val + " KB";
         }
@@ -478,4 +479,159 @@ public final class FileUtils
     {
         fsErrorHandler.getAndSet(handler);
     }
+
+    /**
+     * Returns the size of the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  size overflow.
+     * See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information.</p>
+     *
+     * @param file the partition
+     * @return the size, in bytes, of the partition or {@code 0L} if the abstract pathname does not name a partition
+     */
+    public static long getTotalSpace(File file)
+    {
+        return handleLargeFileSystem(file.getTotalSpace());
+    }
+
+    /**
+     * Returns the number of unallocated bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of unallocated bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of unallocated bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getFreeSpace(File file)
+    {
+        return handleLargeFileSystem(file.getFreeSpace());
+    }
+
+    /**
+     * Returns the number of available bytes on the specified partition.
+     * <p>This method handles large file system by returning {@code Long.MAX_VALUE} if the  number of available bytes
+     * overflow. See <a href='https://bugs.openjdk.java.net/browse/JDK-8179320'>JDK-8179320</a> for more information</p>
+     *
+     * @param file the partition
+     * @return the number of available bytes on the partition or {@code 0L}
+     * if the abstract pathname does not name a partition.
+     */
+    public static long getUsableSpace(File file)
+    {
+        return handleLargeFileSystem(file.getUsableSpace());
+    }
+
+    /**
+     * Returns the {@link FileStore} representing the file store where a file
+     * is located. This {@link FileStore} handles large file system by returning {@code Long.MAX_VALUE}
+     * from {@code FileStore#getTotalSpace()}, {@code FileStore#getUnallocatedSpace()} and {@code FileStore#getUsableSpace()}
+     * it the value is bigger than {@code Long.MAX_VALUE}. See <a href='https://bugs.openjdk.java.net/browse/JDK-8162520'>JDK-8162520</a>
+     * for more information.
+     *
+     * @param path the path to the file
+     * @return the file store where the file is stored
+     */
+    public static FileStore getFileStore(Path path) throws IOException
+    {
+        return new SafeFileStore(Files.getFileStore(path));
+    }
+
+    /**
+     * Handle large file system by returning {@code Long.MAX_VALUE} when the size overflows.
+     * @param size returned by the Java's FileStore methods
+     * @return the size or {@code Long.MAX_VALUE} if the size was bigger than {@code Long.MAX_VALUE}
+     */
+    private static long handleLargeFileSystem(long size)
+    {
+        return size < 0 ? Long.MAX_VALUE : size;
+    }
+
+    /**
+     * Private constructor as the class contains only static methods.
+     */
+    private FileUtils()
+    {
+    }
+
+    /**
+     * FileStore decorator used to safely handle large file system.
+     *
+     * <p>Java's FileStore methods (getTotalSpace/getUnallocatedSpace/getUsableSpace) are limited to reporting bytes as
+     * signed long (2^63-1), if the filesystem is any bigger, then the size overflows. {@code SafeFileStore} will
+     * return {@code Long.MAX_VALUE} if the size overflow.</p>
+     *
+     * @see https://bugs.openjdk.java.net/browse/JDK-8162520.
+     */
+    private static final class SafeFileStore extends FileStore
+    {
+        /**
+         * The decorated {@code FileStore}
+         */
+        private final FileStore fileStore;
+
+        public SafeFileStore(FileStore fileStore)
+        {
+            this.fileStore = fileStore;
+        }
+
+        @Override
+        public String name()
+        {
+            return fileStore.name();
+        }
+
+        @Override
+        public String type()
+        {
+            return fileStore.type();
+        }
+
+        @Override
+        public boolean isReadOnly()
+        {
+            return fileStore.isReadOnly();
+        }
+
+        @Override
+        public long getTotalSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getTotalSpace());
+        }
+
+        @Override
+        public long getUsableSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUsableSpace());
+        }
+
+        @Override
+        public long getUnallocatedSpace() throws IOException
+        {
+            return handleLargeFileSystem(fileStore.getUnallocatedSpace());
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type)
+        {
+            return fileStore.supportsFileAttributeView(type);
+        }
+
+        @Override
+        public boolean supportsFileAttributeView(String name)
+        {
+            return fileStore.supportsFileAttributeView(name);
+        }
+
+        @Override
+        public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type)
+        {
+            return fileStore.getFileStoreAttributeView(type);
+        }
+
+        @Override
+        public Object getAttribute(String attribute) throws IOException
+        {
+            return fileStore.getAttribute(attribute);
+        }
+    }
 }


---------------------------------------------------------------------
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/c0020629
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c0020629
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c0020629

Branch: refs/heads/trunk
Commit: c00206297be451077034daf318eb8912825e135a
Parents: cad9416 303dba6
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 8 17:13:18 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 8 17:13:30 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/config/DatabaseDescriptor.java    |  26 ++-
 .../org/apache/cassandra/db/Directories.java    |   3 +-
 .../org/apache/cassandra/io/util/FileUtils.java | 181 +++++++++++++++++--
 4 files changed, 191 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c0020629/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c0020629/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c0020629/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------


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