You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/06/17 18:04:27 UTC
cassandra git commit: Fix memory leak in Ref due to
ConcurrentLinkedQueue behaviour
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.1 46cc577a8 -> 7c5fc40b8
Fix memory leak in Ref due to ConcurrentLinkedQueue behaviour
patch by benedict; reviewed by marcus for CASSANDRA-9549
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7c5fc40b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7c5fc40b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7c5fc40b
Branch: refs/heads/cassandra-2.1
Commit: 7c5fc40b8b644e05c32479f2581309f75f981421
Parents: 46cc577
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Jun 17 17:02:03 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Jun 17 17:02:03 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/utils/concurrent/Ref.java | 8 +++-----
.../cassandra/utils/concurrent/RefCountedTest.java | 15 +++++++++++++++
3 files changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c5fc40b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f9187c..009d974 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.1.7
+ * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549)
Merged from 2.0
* Periodically submit background compaction tasks (CASSANDRA-9592)
* Set HAS_MORE_PAGES flag to false when PagingState is null (CASSANDRA-9571)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c5fc40b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
index 4e6cef7..f9876de 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
@@ -2,12 +2,10 @@ package org.apache.cassandra.utils.concurrent;
import java.lang.ref.PhantomReference;
import java.lang.ref.ReferenceQueue;
+import java.util.Collection;
import java.util.Collections;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
@@ -233,7 +231,7 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
{
// we need to retain a reference to each of the PhantomReference instances
// we are using to track individual refs
- private final ConcurrentLinkedQueue<State> locallyExtant = new ConcurrentLinkedQueue<>();
+ private final Collection<State> locallyExtant = new ConcurrentLinkedDeque<>();
// the number of live refs
private final AtomicInteger counts = new AtomicInteger();
// the object to call to cleanup when our refs are all finished with
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c5fc40b/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java b/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java
index a9247cd..bb173fe 100644
--- a/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.utils.concurrent;
import org.junit.Test;
import junit.framework.Assert;
+import org.apache.cassandra.utils.ObjectSizes;
public class RefCountedTest
{
@@ -82,4 +83,18 @@ public class RefCountedTest
{
}
}
+
+ @Test
+ public void testMemoryLeak()
+ {
+ Tidier tidier = new Tidier();
+ Ref<Object> ref = new Ref(null, tidier);
+ long initialSize = ObjectSizes.measureDeep(ref);
+ for (int i = 0 ; i < 1000 ; i++)
+ ref.ref().release();
+ long finalSize = ObjectSizes.measureDeep(ref);
+ if (finalSize > initialSize * 2)
+ throw new AssertionError();
+ ref.release();
+ }
}