Repository: cassandra
Updated Branches:
  refs/heads/trunk cdf02f668 -> 6bb757715


Use weak references in compaction logger to avoid strong ref loops

Patch by marcuse; reviewed by Carl Yeksigian for CASSANDRA-12413


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

Branch: refs/heads/trunk
Commit: 6bb7577157e553dc81c280b3c83862e7a397219d
Parents: cdf02f6
Author: Marcus Eriksson <marc...@apache.org>
Authored: Thu Aug 11 09:33:07 2016 +0200
Committer: Marcus Eriksson <marc...@apache.org>
Committed: Fri Sep 2 14:35:52 2016 +0200

----------------------------------------------------------------------
 .../db/compaction/CompactionLogger.java         | 28 ++++++++++++++++----
 1 file changed, 23 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bb75771/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
index 16a7f2a..c8def3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.compaction;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
+import java.lang.ref.WeakReference;
 import java.nio.file.*;
 import java.util.Collection;
 import java.util.HashSet;
@@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import java.util.function.Function;
 
+import com.google.common.collect.MapMaker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -102,20 +104,23 @@ public class CompactionLogger
     private static final JsonNodeFactory json = JsonNodeFactory.instance;
     private static final Logger logger = 
LoggerFactory.getLogger(CompactionLogger.class);
     private static final Writer serializer = new CompactionLogSerializer();
-    private final ColumnFamilyStore cfs;
-    private final CompactionStrategyManager csm;
+    private final WeakReference<ColumnFamilyStore> cfsRef;
+    private final WeakReference<CompactionStrategyManager> csmRef;
     private final AtomicInteger identifier = new AtomicInteger(0);
-    private final Map<AbstractCompactionStrategy, String> 
compactionStrategyMapping = new ConcurrentHashMap<>();
+    private final Map<AbstractCompactionStrategy, String> 
compactionStrategyMapping = new MapMaker().weakKeys().makeMap();
     private final AtomicBoolean enabled = new AtomicBoolean(false);
 
     public CompactionLogger(ColumnFamilyStore cfs, CompactionStrategyManager 
csm)
     {
-        this.csm = csm;
-        this.cfs = cfs;
+        csmRef = new WeakReference<>(csm);
+        cfsRef = new WeakReference<>(cfs);
     }
 
     private void forEach(Consumer<AbstractCompactionStrategy> consumer)
     {
+        CompactionStrategyManager csm = csmRef.get();
+        if (csm == null)
+            return;
         csm.getStrategies()
            .forEach(l -> l.forEach(consumer));
     }
@@ -129,7 +134,10 @@ public class CompactionLogger
 
     private ArrayNode sstableMap(Collection<SSTableReader> sstables, 
CompactionStrategyAndTableFunction csatf)
     {
+        CompactionStrategyManager csm = csmRef.get();
         ArrayNode node = json.arrayNode();
+        if (csm == null)
+            return node;
         sstables.forEach(t -> 
node.add(csatf.apply(csm.getCompactionStrategyFor(t), t)));
         return node;
     }
@@ -142,6 +150,10 @@ public class CompactionLogger
     private JsonNode formatSSTables(AbstractCompactionStrategy strategy)
     {
         ArrayNode node = json.arrayNode();
+        CompactionStrategyManager csm = csmRef.get();
+        ColumnFamilyStore cfs = cfsRef.get();
+        if (csm == null || cfs == null)
+            return node;
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
             if (csm.getCompactionStrategyFor(sstable) == strategy)
@@ -165,6 +177,9 @@ public class CompactionLogger
     private JsonNode startStrategy(AbstractCompactionStrategy strategy)
     {
         ObjectNode node = json.objectNode();
+        CompactionStrategyManager csm = csmRef.get();
+        if (csm == null)
+            return node;
         node.put("strategyId", getId(strategy));
         node.put("type", strategy.getName());
         node.put("tables", formatSSTables(strategy));
@@ -200,6 +215,9 @@ public class CompactionLogger
 
     private void describeStrategy(ObjectNode node)
     {
+        ColumnFamilyStore cfs = cfsRef.get();
+        if (cfs == null)
+            return;
         node.put("keyspace", cfs.keyspace.getName());
         node.put("table", cfs.getTableName());
         node.put("time", System.currentTimeMillis());

Reply via email to