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

jolynch pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 97b47c3b5f845097181130125752bd6efc1e1e47
Merge: 507c6f7 e73d05b
Author: Joseph Lynch <joe.e.ly...@gmail.com>
AuthorDate: Thu Dec 9 10:30:15 2021 -0500

    Merge branch 'cassandra-4.0' into trunk

 CHANGES.txt                                        |   1 +
 conf/cassandra.yaml                                |   5 +
 .../apache/cassandra/cache/AutoSavingCache.java    |   8 +-
 src/java/org/apache/cassandra/config/Config.java   |   2 +
 .../cassandra/config/DatabaseDescriptor.java       |  11 ++
 .../org/apache/cassandra/db/lifecycle/View.java    |   2 +-
 .../org/apache/cassandra/service/CacheService.java |  41 ++++--
 .../test/microbench/CacheLoaderBench.java          | 137 ++++++++++++++++++++
 .../unit/org/apache/cassandra/db/KeyCacheTest.java | 138 ++++++++++++++++++++-
 9 files changed, 332 insertions(+), 13 deletions(-)

diff --cc src/java/org/apache/cassandra/cache/AutoSavingCache.java
index 0e022d4,430161f..d0b897e
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@@ -200,8 -209,9 +200,9 @@@ public class AutoSavingCache<K extends 
                                                + " does not match current 
schema version "
                                                + Schema.instance.getVersion());
  
-                 ArrayDeque<Future<Pair<K, V>>> futures = new 
ArrayDeque<Future<Pair<K, V>>>();
-                 while (in.available() > 0)
+                 ArrayDeque<Future<Pair<K, V>>> futures = new ArrayDeque<>();
+                 long loadByNanos = start + 
TimeUnit.SECONDS.toNanos(DatabaseDescriptor.getCacheLoadTimeout());
 -                while (System.nanoTime() < loadByNanos && in.available() > 0)
++                while (nanoTime() < loadByNanos && in.available() > 0)
                  {
                      //tableId and indexName are serialized by the serializers 
in CacheService
                      //That is delegated there because there are serializer 
specific conditions
diff --cc src/java/org/apache/cassandra/config/Config.java
index 9fb57797,c03bd96..f08da7a
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@@ -332,8 -311,8 +332,10 @@@ public class Confi
      public volatile int counter_cache_save_period = 7200;
      public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;
  
+     public int cache_load_timeout_seconds = 30;
+ 
 +    public Long paxos_cache_size_in_mb = null;
 +
      private static boolean isClientMode = false;
      private static Supplier<Config> overrideLoadConfig = null;
  
diff --cc src/java/org/apache/cassandra/service/CacheService.java
index 3636c13,0a281ad..1725d5f
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@@ -20,10 -20,16 +20,13 @@@ package org.apache.cassandra.service
  import java.io.IOException;
  import java.nio.ByteBuffer;
  import java.util.ArrayList;
+ import java.util.HashMap;
  import java.util.Iterator;
  import java.util.List;
+ import java.util.Map;
  import java.util.concurrent.Callable;
+ import java.util.concurrent.ConcurrentHashMap;
  import java.util.concurrent.ExecutionException;
 -import java.util.concurrent.Future;
 -
 -import com.google.common.util.concurrent.Futures;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -456,17 -485,12 +484,12 @@@ public class CacheService implements Ca
                                                                                
                                  reader.descriptor.version,
                                                                                
                                  reader.header);
              RowIndexEntry<?> entry = 
indexSerializer.deserializeForCache(input);
 -            return Futures.immediateFuture(Pair.create(new 
KeyCacheKey(cfs.metadata(), reader.descriptor, key), entry));
 +            return ImmediateFuture.success(Pair.create(new 
KeyCacheKey(cfs.metadata(), reader.descriptor, key), entry));
          }
  
-         private SSTableReader findDesc(int generation, 
Iterable<SSTableReader> collection)
+         public void cleanupAfterDeserialize()
          {
-             for (SSTableReader sstable : collection)
-             {
-                 if (sstable.descriptor.generation == generation)
-                     return sstable;
-             }
-             return null;
+             cachedSSTableReaders.clear();
          }
      }
  }

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

Reply via email to