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

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

commit efce6b39fb557314fad0cb56b07a1919d26b84fc
Merge: 139ad44 ecd23f1
Author: Sylvain Lebresne <lebre...@gmail.com>
AuthorDate: Mon Aug 17 11:39:24 2020 +0200

    Merge branch 'cassandra-3.11' into trunk

 CHANGES.txt                                        |  1 +
 .../cassandra/db/SinglePartitionReadCommand.java   | 25 ----------------------
 .../db/compaction/AbstractCompactionStrategy.java  |  5 -----
 .../db/compaction/CompactionStrategyManager.java   |  7 ------
 .../compaction/SizeTieredCompactionStrategy.java   |  6 ------
 5 files changed, 1 insertion(+), 43 deletions(-)

diff --cc CHANGES.txt
index 58239c9,a6bc9d9..7298e0c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,14 +1,21 @@@
 -3.11.8
 +4.0-beta2
 + * Make TimestampSerializer accept fractional seconds of varying precision 
(CASSANDRA-15976)
 + * Improve cassandra-stress logging when using a profile file that doesn't 
exist (CASSANDRA-14425)
 + * Improve logging for socket connection/disconnection (CASSANDRA-15980)
 + * Throw FSWriteError upon write failures in order to apply DiskFailurePolicy 
(CASSANDRA-15928)
 + * Forbid altering UDTs used in partition keys (CASSANDRA-15933)
 + * Fix version parsing logic when upgrading from 3.0 (CASSANDRA-15973)
 + * Optimize NoSpamLogger use in hot paths (CASSANDRA-15766)
 + * Verify sstable components on startup (CASSANDRA-15945)
 +Merged from 3.11:
   * Fix short read protection for GROUP BY queries (CASSANDRA-15459)
 + * stop_paranoid disk failure policy is ignored on CorruptSSTableException 
after node is up (CASSANDRA-15191)
   * Frozen RawTuple is not annotated with frozen in the toString method 
(CASSANDRA-15857)
  Merged from 3.0:
+  * Remove broken 'defrag-on-read' optimization (CASSANDRA-15432)
   * Check for endpoint collision with hibernating nodes (CASSANDRA-14599)
   * Operational improvements and hardening for replica filtering protection 
(CASSANDRA-15907)
 - * stop_paranoid disk failure policy is ignored on CorruptSSTableException 
after node is up (CASSANDRA-15191)
 - * Forbid altering UDTs used in partition keys (CASSANDRA-15933)
   * Fix empty/null json string representation (CASSANDRA-15896)
 - * 3.x fails to start if commit log has range tombstones from a column which 
is also deleted (CASSANDRA-15970)
  Merged from 2.2:
   * Fix CQL parsing of collections when the column type is reversed 
(CASSANDRA-15814)
  
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 6fbe523,c5de444..e581be5
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -28,10 -28,15 +28,9 @@@ import com.google.common.collect.Sets
  import org.apache.cassandra.cache.IRowCacheEntry;
  import org.apache.cassandra.cache.RowCacheKey;
  import org.apache.cassandra.cache.RowCacheSentinel;
--import org.apache.cassandra.concurrent.Stage;
 -import org.apache.cassandra.concurrent.StageManager;
 -import org.apache.cassandra.config.CFMetaData;
 -import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.db.lifecycle.*;
  import org.apache.cassandra.db.filter.*;
 +import org.apache.cassandra.db.lifecycle.*;
  import org.apache.cassandra.db.partitions.*;
  import org.apache.cassandra.db.rows.*;
  import org.apache.cassandra.db.transform.RTBoundValidator;
@@@ -889,11 -992,8 +887,8 @@@ public class SinglePartitionReadComman
                  if (iter.isEmpty())
                      continue;
  
-                 if (sstable.isRepaired())
-                     onlyUnrepaired = false;
- 
                  result = add(
 -                    RTBoundValidator.validate(isForThrift() ? 
ThriftResultsMerger.maybeWrap(iter, nowInSec()) : iter, 
RTBoundValidator.Stage.SSTABLE, false),
 +                    RTBoundValidator.validate(iter, 
RTBoundValidator.Stage.SSTABLE, false),
                      result,
                      filter,
                      sstable.isRepaired()
@@@ -907,29 -1007,9 +902,9 @@@
              return EmptyIterators.unfilteredRow(metadata(), partitionKey(), 
false);
  
          DecoratedKey key = result.partitionKey();
 -        
cfs.metric.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), 
key.hashCode(), 1);
 -        StorageHook.instance.reportRead(cfs.metadata.cfId, partitionKey());
 +        cfs.metric.topReadPartitionFrequency.addSample(key.getKey(), 1);
 +        StorageHook.instance.reportRead(cfs.metadata.id, partitionKey());
  
-         // "hoist up" the requested data into a more recent sstable
-         if (metricsCollector.getMergedSSTables() > 
cfs.getMinimumCompactionThreshold()
-             && onlyUnrepaired
-             && !cfs.isAutoCompactionDisabled()
-             && cfs.getCompactionStrategyManager().shouldDefragment())
-         {
-             // !!WARNING!!   if we stop copying our data to a heap-managed 
object,
-             //               we will need to track the lifetime of this 
mutation as well
-             Tracing.trace("Defragmenting requested data");
- 
-             try (UnfilteredRowIterator iter = 
result.unfilteredIterator(columnFilter(), Slices.ALL, false))
-             {
-                 final Mutation mutation = new 
Mutation(PartitionUpdate.fromIterator(iter, columnFilter()));
-                 Stage.MUTATION.execute(() -> {
-                     // skipping commitlog and index updates is fine since 
we're just de-fragmenting existing data
-                     Keyspace.open(mutation.getKeyspaceName()).apply(mutation, 
false, false);
-                 });
-             }
-         }
- 
          return result.unfilteredIterator(columnFilter(), Slices.ALL, 
clusteringIndexFilter().isReversed());
      }
  
diff --cc 
src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 546f61b,d486679..708faff
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@@ -132,9 -102,8 +132,8 @@@ public class CompactionStrategyManager 
  
          If a user changes the local compaction strategy and then later ALTERs 
a compaction parameter,
          we will use the new compaction parameters.
 -     **/
 +     */
      private volatile CompactionParams schemaCompactionParams;
-     private boolean shouldDefragment;
      private boolean supportsEarlyOpen;
      private int fanout;
  
@@@ -308,10 -213,10 +307,9 @@@
                  if (sstable.openReason != SSTableReader.OpenReason.EARLY)
                      compactionStrategyFor(sstable).addSSTable(sstable);
              }
 -            repaired.forEach(AbstractCompactionStrategy::startup);
 -            unrepaired.forEach(AbstractCompactionStrategy::startup);
 -            supportsEarlyOpen = repaired.get(0).supportsEarlyOpen();
 -            fanout = (repaired.get(0) instanceof LeveledCompactionStrategy) ? 
((LeveledCompactionStrategy) repaired.get(0)).getLevelFanoutSize() : 
LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
 +            holders.forEach(AbstractStrategyHolder::startup);
-             shouldDefragment = repaired.first().shouldDefragment();
 +            supportsEarlyOpen = repaired.first().supportsEarlyOpen();
 +            fanout = (repaired.first() instanceof LeveledCompactionStrategy) 
? ((LeveledCompactionStrategy) repaired.first()).getLevelFanoutSize() : 
LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
          }
          finally
          {
@@@ -600,11 -470,21 +598,6 @@@
          return res;
      }
  
-     public boolean shouldDefragment()
 -    public Directories getDirectories()
--    {
-         return shouldDefragment;
 -        maybeReloadDiskBoundaries();
 -        readLock.lock();
 -        try
 -        {
 -            assert 
repaired.get(0).getClass().equals(unrepaired.get(0).getClass());
 -            return repaired.get(0).getDirectories();
 -        }
 -        finally
 -        {
 -            readLock.unlock();
 -        }
--    }
--
      private void handleFlushNotification(Iterable<SSTableReader> added)
      {
          // If reloaded, SSTables will be placed in their correct locations


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

Reply via email to