Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1.0 d7f7eec27 -> e9d0214a1


Fix NPE in FileCacheService.sizeInBytes

patch by tjake; reviewed by jbellis for (CASSANDRA-7756)


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

Branch: refs/heads/cassandra-2.1.0
Commit: ad6ba3d243058f060569ad16d6713f46e2ce6160
Parents: e756660
Author: Jake Luciani <j...@apache.org>
Authored: Fri Aug 15 13:24:15 2014 -0400
Committer: Jake Luciani <j...@apache.org>
Committed: Fri Aug 15 13:24:15 2014 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/io/util/RandomAccessReader.java   |  5 +-
 .../io/util/BufferedRandomAccessFileTest.java   | 70 ++++++++++++++++++++
 3 files changed, 75 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad6ba3d2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 04902ad..4306de5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.10
+ * Fix NPE in FileCacheService.sizeInBytes (CASSANDRA-7756)
  * (cqlsh) cqlsh should automatically disable tracing when selecting
    from system_traces (CASSANDRA-7641)
  * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad6ba3d2/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java 
b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index 9a03480..09ecac0 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -154,7 +154,10 @@ public class RandomAccessReader extends RandomAccessFile 
implements FileDataInpu
 
     public int getTotalBufferSize()
     {
-        return buffer.length;
+        //This may NPE so we make a ref
+        //https://issues.apache.org/jira/browse/CASSANDRA-7756
+        byte[] ref = buffer;
+        return ref != null ? ref.length : 0;
     }
 
     public void reset()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad6ba3d2/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git 
a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java 
b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index 90c27e3..a16b291 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -19,6 +19,7 @@
  */
 package org.apache.cassandra.io.util;
 
+import org.apache.cassandra.service.FileCacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import java.io.File;
@@ -28,6 +29,11 @@ import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.cassandra.Util.expectEOF;
 import static org.apache.cassandra.Util.expectException;
@@ -508,6 +514,70 @@ public class BufferedRandomAccessFileTest
     }
 
     @Test
+    public void testFileCacheService() throws IOException, InterruptedException
+    {
+        //see https://issues.apache.org/jira/browse/CASSANDRA-7756
+
+        final int THREAD_COUNT = 40;
+        ExecutorService executorService = 
Executors.newFixedThreadPool(THREAD_COUNT);
+
+        SequentialWriter w1 = createTempFile("fscache1");
+        SequentialWriter w2 = createTempFile("fscache2");
+
+        w1.write(new byte[30]);
+        w1.close();
+
+        w2.write(new byte[30]);
+        w2.close();
+
+        for (int i = 0; i < 20; i++)
+        {
+
+
+            RandomAccessReader r1 = RandomAccessReader.open(w1);
+            RandomAccessReader r2 = RandomAccessReader.open(w2);
+
+
+            FileCacheService.instance.put(r1);
+            FileCacheService.instance.put(r2);
+
+            final CountDownLatch finished = new CountDownLatch(THREAD_COUNT);
+            final AtomicBoolean hadError = new AtomicBoolean(false);
+
+            for (int k = 0; k < THREAD_COUNT; k++)
+            {
+                executorService.execute( new Runnable()
+                {
+                    @Override
+                    public void run()
+                    {
+                        try
+                        {
+                            long size = 
FileCacheService.instance.sizeInBytes();
+
+                            while (size > 0)
+                                size = FileCacheService.instance.sizeInBytes();
+                        }
+                        catch (Throwable t)
+                        {
+                            t.printStackTrace();
+                            hadError.set(true);
+                        }
+                        finally
+                        {
+                            finished.countDown();
+                        }
+                    }
+                });
+
+            }
+
+            finished.await();
+            assert !hadError.get();
+        }
+    }
+
+    @Test
     public void testReadOnly() throws IOException
     {
         SequentialWriter file = createTempFile("brafReadOnlyTest");

Reply via email to