This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch cassandra-4.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit a2955d9b3405deadca4d754d7f061c85e7247896
Merge: 5adc310 b5f0f4c
Author: Brandon Williams <brandonwilli...@apache.org>
AuthorDate: Mon Jul 19 07:34:11 2021 -0500

    Merge branch 'cassandra-3.11' into cassandra-4.0

 CHANGES.txt                                        |   1 +
 .../io/sstable/format/big/BigTableReader.java      |  14 ++-
 .../cassandra/io/sstable/SSTableReaderTest.java    | 113 ++++++++++++++++-----
 3 files changed, 100 insertions(+), 28 deletions(-)

diff --cc CHANGES.txt
index 2fe22af,a43559e..b51f94f
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -9,35 -3,6 +9,36 @@@ Merged from 3.11
   * Optimize bytes skipping when reading SSTable files (CASSANDRA-14415)
   * Enable tombstone compactions when unchecked_tombstone_compaction is set in 
TWCS (CASSANDRA-14496)
   * Read only the required SSTables for single partition queries 
(CASSANDRA-16737)
 +Merged from 3.0:
++ * Count bloom filter misses correctly (CASSANDRA-12922)
 + * Reject token() in MV WHERE clause (CASSANDRA-13464)
 + * Ensure java executable is on the path (CASSANDRA-14325)
 + * Clean transaction log leftovers at the beginning of sstablelevelreset and 
sstableofflinerelevel (CASSANDRA-12519)
 +
 +4.0.0
 + * Fix fwd to/from headers in DC write forwarding (CASSANDRA-16797)
 + * Fix CassandraVersion::compareTo (CASSANDRA-16794)
 + * BinLog does not close chronicle queue leaving this to GC to cleanup 
(CASSANDRA-16774)
 +Merged from 3.11:
 +Merged from 3.0:
 +
 +4.0-rc2
 + * Avoid memoizing the wrong min cluster version during upgrades 
(CASSANDRA-16759)
 + * Obfuscate passwords in statements in QueryEvents (CASSANDRA-16669)
 + * Fix queries on empty partitions with static data (CASSANDRA-16686)
 + * Keep python driver in artifacts (CASSANDRA-16700)
 + * Improve AuditLogging documentation and logback.xml(CASSANDRA-16682)
 + * Spin up SEPWorker threads whenever we grow the number of work 
permits(CASSANDRA-16668)
 + * Add a warning to cqlsh 6.0.0 that DESCRIBE does not work with a Cassandra 
3.x servers (CASSANDRA-16652)
 + * cqlsh: fix DESC TYPE with non-ascii character in the identifier 
(CASSANDRA-16400)
 + * Remove drivers dependency and bring cql_keywords_reserved on server side 
(CASSANDRA-16659)
 + * Fix in-browser "help", Python 3 (CASSANDRA-16658)
 + * Fix DROP COMPACT STORAGE for counters (CASSANDRA-16653)
 + * Add back validation for AlterTableStatements (CASSANDRA-16643)
 + * Fix cqlsh encoding error with unicode in multi-line statement 
(CASSANDRA-16539)
 + * Fix race in fat client removal (CASSANDRA-16238)
 + * Test org.apache.cassandra.net.AsyncPromiseTest FAILED (CASSANDRA-16596)
 +Merged from 3.11:
   * Fix LeveledCompactionStrategy compacts last level throw an 
ArrayIndexOutOfBoundsException (CASSANDRA-15669)
   * Maps $CASSANDRA_LOG_DIR to cassandra.logdir java property when executing 
nodetool (CASSANDRA-16199)
   * Nodetool garbagecollect should retain SSTableLevel for LCS 
(CASSANDRA-16634)
diff --cc 
src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
index f60c9df,db28bcd..dc13031
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
@@@ -154,10 -156,13 +161,12 @@@ public class BigTableReader extends SST
          // next, the key cache (only make sense for valid row key)
          if ((op == Operator.EQ || op == Operator.GE) && (key instanceof 
DecoratedKey))
          {
 -            DecoratedKey decoratedKey = (DecoratedKey)key;
 -            KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, 
descriptor, decoratedKey.getKey());
 -            RowIndexEntry cachedPosition = getCachedPosition(cacheKey, 
updateCacheAndStats);
 +            DecoratedKey decoratedKey = (DecoratedKey) key;
 +            RowIndexEntry cachedPosition = getCachedPosition(decoratedKey, 
updateCacheAndStats);
              if (cachedPosition != null)
              {
+                 // we do not need to track "true positive" for Bloom Filter 
here because it has been already tracked
+                 // inside getCachedPosition method
                  listener.onSSTableSelected(this, cachedPosition, 
SelectionReason.KEY_CACHE_HIT);
                  Tracing.trace("Key cache hit for sstable {}", 
descriptor.generation);
                  return cachedPosition;
diff --cc test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 0b64028,d225801..c905732
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@@ -72,9 -67,9 +72,10 @@@ public class SSTableReaderTes
      public static final String KEYSPACE1 = "SSTableReaderTest";
      public static final String CF_STANDARD = "Standard1";
      public static final String CF_STANDARD2 = "Standard2";
 +    public static final String CF_COMPRESSED = "Compressed";
      public static final String CF_INDEXED = "Indexed1";
-     public static final String CF_STANDARDLOWINDEXINTERVAL = 
"StandardLowIndexInterval";
+     public static final String CF_STANDARD_LOW_INDEX_INTERVAL = 
"StandardLowIndexInterval";
+     public static final String CF_STANDARD_SMALL_BLOOM_FILTER = 
"StandardSmallBloomFilter";
  
      private IPartitioner partitioner;
  
@@@ -91,12 -86,15 +92,16 @@@
                                      KeyspaceParams.simple(1),
                                      SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARD),
                                      SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARD2),
 +                                    SchemaLoader.standardCFMD(KEYSPACE1, 
CF_COMPRESSED).compression(CompressionParams.DEFAULT),
                                      
SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEXED, true),
-                                     SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARDLOWINDEXINTERVAL)
+                                     SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARD_LOW_INDEX_INTERVAL)
                                                  .minIndexInterval(8)
                                                  .maxIndexInterval(256)
-                                                 
.caching(CachingParams.CACHE_NOTHING));
+                                                 
.caching(CachingParams.CACHE_NOTHING),
+                                     SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARD_SMALL_BLOOM_FILTER)
+                                                 .minIndexInterval(4)
+                                                 .maxIndexInterval(4)
+                                                 .bloomFilterFpChance(0.99));
      }
  
      @Test
@@@ -324,18 -324,74 +331,74 @@@
          CompactionManager.instance.performMaximal(store, false);
  
          SSTableReader sstable = store.getLiveSSTables().iterator().next();
+         // existing, non-cached key
          sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(1, sstable.getKeyCacheRequest());
          assertEquals(0, sstable.getKeyCacheHit());
-         assertEquals(1, sstable.getBloomFilterTruePositiveCount());
+         // existing, cached key
          sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getKeyCacheRequest());
          assertEquals(1, sstable.getKeyCacheHit());
-         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
-         sstable.getPosition(k(15), SSTableReader.Operator.EQ);
+         // non-existing key (it is specifically chosen to not be rejected by 
Bloom Filter check)
+         sstable.getPosition(k(14), SSTableReader.Operator.EQ);
+         assertEquals(3, sstable.getKeyCacheRequest());
          assertEquals(1, sstable.getKeyCacheHit());
-         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
- 
      }
  
+     @Test
+     public void testGetPositionsBloomFilterStats()
+     {
+         Keyspace keyspace = Keyspace.open(KEYSPACE1);
+         ColumnFamilyStore store = 
keyspace.getColumnFamilyStore(CF_STANDARD_SMALL_BLOOM_FILTER);
+         partitioner = store.getPartitioner();
+         CacheService.instance.keyCache.setCapacity(1000);
+ 
+         // insert data and compact to a single sstable
+         CompactionManager.instance.disableAutoCompaction();
+         for (int j = 0; j < 10; j++)
+         {
 -            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
++            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
+                     .clustering("0")
+                     .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+                     .build()
+                     .applyUnsafe();
+         }
+         store.forceBlockingFlush();
+         CompactionManager.instance.performMaximal(store, false);
+ 
+         SSTableReader sstable = store.getLiveSSTables().iterator().next();
+         // the keys are specifically chosen to cover certain use cases
+         // existing key is read from index
+         sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(1, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(0, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(0, sstable.getBloomFilterFalsePositiveCount());
+         // existing key is read from Cache Key
+         sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(0, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(0, sstable.getBloomFilterFalsePositiveCount());
+         // non-existing key is rejected by Bloom Filter check
+         sstable.getPosition(k(10), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(1, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(0, sstable.getBloomFilterFalsePositiveCount());
+         // non-existing key is rejected by sstable keys range check
+         sstable.getPosition(k(99), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(1, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(1, sstable.getBloomFilterFalsePositiveCount());
+         // non-existing key is rejected by index interval check
+         sstable.getPosition(k(14), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(1, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(2, sstable.getBloomFilterFalsePositiveCount());
+         // non-existing key is rejected by index lookup check
+         sstable.getPosition(k(807), SSTableReader.Operator.EQ);
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         assertEquals(1, sstable.getBloomFilterTrueNegativeCount());
+         assertEquals(3, sstable.getBloomFilterFalsePositiveCount());
+     }
  
      @Test
      public void testOpeningSSTable() throws Exception
@@@ -462,9 -521,9 +525,9 @@@
      public void testLoadingSummaryUsesCorrectPartitioner() throws Exception
      {
          Keyspace keyspace = Keyspace.open(KEYSPACE1);
-         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
+         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_INDEXED);
  
 -        new RowUpdateBuilder(store.metadata, System.currentTimeMillis(), "k1")
 +        new RowUpdateBuilder(store.metadata(), System.currentTimeMillis(), 
"k1")
          .clustering("0")
          .add("birthdate", 1L)
          .build()
@@@ -490,10 -549,10 +553,10 @@@
      public void testGetScannerForNoIntersectingRanges() throws Exception
      {
          Keyspace keyspace = Keyspace.open(KEYSPACE1);
-         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD);
          partitioner = store.getPartitioner();
  
 -        new RowUpdateBuilder(store.metadata, 0, "k1")
 +        new RowUpdateBuilder(store.metadata(), 0, "k1")
              .clustering("xyz")
              .add("val", "abc")
              .build()
@@@ -699,150 -758,4 +762,150 @@@
      {
          return new BufferDecoratedKey(t(i), 
ByteBufferUtil.bytes(String.valueOf(i)));
      }
 +
 +    @Test(expected = RuntimeException.class)
 +    public void testMoveAndOpenLiveSSTable()
 +    {
 +        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD);
 +        SSTableReader sstable = getNewSSTable(cfs);
 +        Descriptor notLiveDesc = new Descriptor(new File("/tmp"), "", "", 0);
 +        SSTableReader.moveAndOpenSSTable(cfs, sstable.descriptor, 
notLiveDesc, sstable.components, false);
 +    }
 +
 +    @Test(expected = RuntimeException.class)
 +    public void testMoveAndOpenLiveSSTable2()
 +    {
 +        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD);
 +        SSTableReader sstable = getNewSSTable(cfs);
 +        Descriptor notLiveDesc = new Descriptor(new File("/tmp"), "", "", 0);
 +        SSTableReader.moveAndOpenSSTable(cfs, notLiveDesc, 
sstable.descriptor, sstable.components, false);
 +    }
 +
 +    @Test
 +    public void testMoveAndOpenSSTable() throws IOException
 +    {
 +        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD);
 +        SSTableReader sstable = getNewSSTable(cfs);
 +        cfs.clearUnsafe();
 +        sstable.selfRef().release();
 +        File tmpdir = Files.createTempDirectory("testMoveAndOpen").toFile();
 +        tmpdir.deleteOnExit();
 +        Descriptor notLiveDesc = new Descriptor(tmpdir, 
sstable.descriptor.ksname, sstable.descriptor.cfname, 100);
 +        // make sure the new directory is empty and that the old files exist:
 +        for (Component c : sstable.components)
 +        {
 +            File f = new File(notLiveDesc.filenameFor(c));
 +            assertFalse(f.exists());
 +            assertTrue(new File(sstable.descriptor.filenameFor(c)).exists());
 +        }
 +        SSTableReader.moveAndOpenSSTable(cfs, sstable.descriptor, 
notLiveDesc, sstable.components, false);
 +        // make sure the files were moved:
 +        for (Component c : sstable.components)
 +        {
 +            File f = new File(notLiveDesc.filenameFor(c));
 +            assertTrue(f.exists());
 +            assertTrue(f.toString().contains("-100-"));
 +            f.deleteOnExit();
 +            assertFalse(new File(sstable.descriptor.filenameFor(c)).exists());
 +        }
 +    }
 +
 +
 +
 +    private SSTableReader getNewSSTable(ColumnFamilyStore cfs)
 +    {
 +
 +        Set<SSTableReader> before = cfs.getLiveSSTables();
 +        for (int j = 0; j < 100; j += 2)
 +        {
 +            new RowUpdateBuilder(cfs.metadata(), j, String.valueOf(j))
 +            .clustering("0")
 +            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
 +            .build()
 +            .applyUnsafe();
 +        }
 +        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");
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD);
 +        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));
 +        }
 +    }
 +
 +    @Rule
 +    public ExpectedException expectedException = ExpectedException.none();
 +
 +    @Test
 +    public void testVerifyCompressionInfoExistenceThrows()
 +    {
 +        Descriptor desc = setUpForTestVerfiyCompressionInfoExistence();
 +
 +        // delete the compression info, so it is corrupted.
 +        File compressionInfoFile = new 
File(desc.filenameFor(Component.COMPRESSION_INFO));
 +        compressionInfoFile.delete();
 +        assertFalse("CompressionInfo file should not exist", 
compressionInfoFile.exists());
 +
 +        // discovert the components on disk after deletion
 +        Set<Component> components = SSTable.discoverComponentsFor(desc);
 +
 +        expectedException.expect(CorruptSSTableException.class);
 +        expectedException.expectMessage("CompressionInfo.db");
 +        SSTableReader.verifyCompressionInfoExistenceIfApplicable(desc, 
components);
 +    }
 +
 +    @Test
 +    public void testVerifyCompressionInfoExistenceWhenTOCUnableToOpen()
 +    {
 +        Descriptor desc = setUpForTestVerfiyCompressionInfoExistence();
 +        Set<Component> components = SSTable.discoverComponentsFor(desc);
 +        SSTableReader.verifyCompressionInfoExistenceIfApplicable(desc, 
components);
 +
 +        // mark the toc file not readable in order to trigger the FSReadError
 +        File tocFile = new File(desc.filenameFor(Component.TOC));
 +        tocFile.setReadable(false);
 +
 +        expectedException.expect(FSReadError.class);
 +        expectedException.expectMessage("TOC.txt");
 +        SSTableReader.verifyCompressionInfoExistenceIfApplicable(desc, 
components);
 +    }
 +
 +    @Test
 +    public void testVerifyCompressionInfoExistencePasses()
 +    {
 +        Descriptor desc = setUpForTestVerfiyCompressionInfoExistence();
 +        Set<Component> components = SSTable.discoverComponentsFor(desc);
 +        SSTableReader.verifyCompressionInfoExistenceIfApplicable(desc, 
components);
 +    }
 +
 +    private Descriptor setUpForTestVerfiyCompressionInfoExistence()
 +    {
 +        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_COMPRESSED);
 +        SSTableReader sstable = getNewSSTable(cfs);
 +        cfs.clearUnsafe();
 +        Descriptor desc = sstable.descriptor;
 +
 +        File compressionInfoFile = new 
File(desc.filenameFor(Component.COMPRESSION_INFO));
 +        File tocFile = new File(desc.filenameFor(Component.TOC));
 +        assertTrue("CompressionInfo file should exist", 
compressionInfoFile.exists());
 +        assertTrue("TOC file should exist", tocFile.exists());
 +        return desc;
 +    }
  }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to