You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2019/05/09 12:36:09 UTC
[cassandra] branch trunk updated: Take sstable references before
calculating approximate key count
This is an automated email from the ASF dual-hosted git repository.
marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push:
new ded6207 Take sstable references before calculating approximate key count
ded6207 is described below
commit ded62076e7fdfd1cfdcf96447489ea607ca796a0
Author: nvharikrishna <n....@gmail.com>
AuthorDate: Fri Mar 1 17:24:19 2019 +0530
Take sstable references before calculating approximate key count
Patch by Venkata Harikrishna Nukala; reviewed by marcuse for CASSANDRA-14647
---
CHANGES.txt | 1 +
.../cassandra/io/sstable/format/SSTableReader.java | 2 +-
.../org/apache/cassandra/metrics/TableMetrics.java | 7 ++++++-
.../cassandra/io/sstable/SSTableReaderTest.java | 23 ++++++++++++++++++++--
4 files changed, 29 insertions(+), 4 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 54ffe5b..e4849d3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
4.0
+ * Take sstable references before calculating approximate key count (CASSANDRA-14647)
* Restore snapshotting of system keyspaces on version change (CASSANDRA-14412)
* Fix AbstractBTreePartition locking in java 11 (CASSANDRA-14607)
* SimpleClient should pass connection properties as options (CASSANDRA-15056)
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 01e30d6..ca6eb85 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -26,7 +26,6 @@ import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterables;
@@ -293,6 +292,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
// if something went wrong above or cardinality is not available, calculate using index summary
if (count < 0)
{
+ count = 0;
for (SSTableReader sstable : sstables)
count += sstable.estimatedKeys();
}
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index 41771d1..c854c43 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.Memtable;
import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.index.SecondaryIndexManager;
import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -505,7 +506,11 @@ public class TableMetrics
long memtablePartitions = 0;
for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
memtablePartitions += memtable.partitionCount();
- return SSTableReader.getApproximateKeyCount(cfs.getSSTables(SSTableSet.CANONICAL)) + memtablePartitions;
+ try(ColumnFamilyStore.RefViewFragment refViewFragment = cfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL)))
+ {
+ return SSTableReader.getApproximateKeyCount(refViewFragment.sstables) + memtablePartitions;
+ }
+
}
});
estimatedColumnCountHistogram = Metrics.register(factory.createMetricName("EstimatedColumnCountHistogram"),
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index c155842..580b099 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -27,7 +27,6 @@ import java.util.*;
import java.util.concurrent.*;
import com.google.common.collect.Sets;
-import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -40,6 +39,8 @@ import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
import org.apache.cassandra.db.rows.Row;
import org.apache.cassandra.dht.IPartitioner;
@@ -55,8 +56,8 @@ import org.apache.cassandra.schema.KeyspaceParams;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FilterFactory;
-import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
+import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
@@ -761,4 +762,22 @@ public class SSTableReaderTest
cfs.forceBlockingFlush();
return Sets.difference(cfs.getLiveSSTables(), before).iterator().next();
}
+
+ @Test
+ public void testGetApproximateKeyCount() throws InterruptedException
+ {
+ Keyspace keyspace = Keyspace.open(KEYSPACE1);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+ cfs.discardSSTables(System.currentTimeMillis()); //Cleaning all existing SSTables.
+ getNewSSTable(cfs);
+
+ try (ColumnFamilyStore.RefViewFragment viewFragment1 = cfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL)))
+ {
+ cfs.discardSSTables(System.currentTimeMillis());
+
+ TimeUnit.MILLISECONDS.sleep(1000); //Giving enough time to clear files.
+ List<SSTableReader> sstables = new ArrayList<>(viewFragment1.sstables);
+ assertEquals(50, SSTableReader.getApproximateKeyCount(sstables));
+ }
+ }
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org