cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bened...@apache.org
Subject [1/3] cassandra git commit: Do not load read meters for offline operations
Date Fri, 03 Apr 2015 11:58:20 GMT
Repository: cassandra
Updated Branches:
  refs/heads/trunk c2ecfe7b7 -> 23c84b169


Do not load read meters for offline operations

patch by benedict; reviewed by tyler for CASSANDRA-9082


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

Branch: refs/heads/trunk
Commit: 345455dee2b154e5a9b10a7a615bcc0c7092775d
Parents: 49d64c2
Author: Benedict Elliott Smith <benedict@apache.org>
Authored: Fri Apr 3 12:53:45 2015 +0100
Committer: Benedict Elliott Smith <benedict@apache.org>
Committed: Fri Apr 3 12:53:45 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/io/sstable/SSTableReader.java     | 24 ++++++++++++++------
 2 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/345455de/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1499c1..9ddb9c9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.5
+ * Do not load read meter for offline operations (CASSANDRA-9082)
  * cqlsh: Make CompositeType data readable (CASSANDRA-8919)
  * cqlsh: Fix display of triggers (CASSANDRA-9081)
  * Fix NullPointerException when deleting or setting an element by index on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/345455de/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index 8fd7b85..c73d4a1 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -378,6 +378,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         return open(descriptor, components, metadata, partitioner, true);
     }
 
+    // use only for offline or "Standalone" operations
     public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component>
components, CFMetaData metadata) throws IOException
     {
         return open(descriptor, components, metadata, StorageService.getPartitioner(), false);
@@ -434,7 +435,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
         sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
         sstable.bf = FilterFactory.AlwaysPresent;
-        sstable.setup();
+        sstable.setup(true);
         return sstable;
     }
 
@@ -478,7 +479,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         sstable.load(validationMetadata);
         logger.debug("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime()
- start));
 
-        sstable.setup();
+        sstable.setup(!validate);
         if (validate)
             sstable.validate();
 
@@ -599,7 +600,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         this.dfile = dfile;
         this.indexSummary = indexSummary;
         this.bf = bloomFilter;
-        this.setup();
+        this.setup(false);
     }
 
     public static long getTotalBytes(Iterable<SSTableReader> sstables)
@@ -2010,9 +2011,9 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         return selfRef.ref();
     }
 
-    void setup()
+    void setup(boolean isOffline)
     {
-        tidy.setup(this);
+        tidy.setup(this, isOffline);
         this.readMeter = tidy.global.readMeter;
     }
 
@@ -2059,7 +2060,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
 
         private boolean setup;
 
-        void setup(SSTableReader reader)
+        void setup(SSTableReader reader, boolean isOffline)
         {
             this.setup = true;
             this.bf = reader.bf;
@@ -2070,6 +2071,8 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             this.typeRef = DescriptorTypeTidy.get(reader);
             this.type = typeRef.get();
             this.global = type.globalRef.get();
+            if (!isOffline)
+                global.ensureReadMeter();
         }
 
         InstanceTidier(Descriptor descriptor, CFMetaData metadata)
@@ -2212,7 +2215,7 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
         private RestorableMeter readMeter;
         // the scheduled persistence of the readMeter, that we will cancel once all instances
of this logical
         // sstable have been released
-        private final ScheduledFuture readMeterSyncFuture;
+        private ScheduledFuture readMeterSyncFuture;
         // shared state managing if the logical sstable has been compacted; this is used
in cleanup both here
         // and in the FINAL type tidier
         private final AtomicBoolean isCompacted;
@@ -2222,6 +2225,13 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             this.desc = reader.descriptor;
             this.isCompacted = new AtomicBoolean();
             this.live = reader;
+        }
+
+        void ensureReadMeter()
+        {
+            if (readMeter != null)
+                return;
+
             // Don't track read rates for tables in the system keyspace and don't bother
trying to load or persist
             // the read meter when in client mode.
             if (Keyspace.SYSTEM_KS.equals(desc.ksname) || Config.isClientMode())


Mime
View raw message