- 
                Notifications
    
You must be signed in to change notification settings  - Fork 3.8k
 
CASSANDRA-20383 #4428
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: cep-45-mutation-tracking
Are you sure you want to change the base?
CASSANDRA-20383 #4428
Changes from 1 commit
60cbed1
              7dbf5e1
              e357277
              8030308
              03a1a47
              435d467
              460aa45
              1516ad6
              0dd99c7
              21e0f35
              028ddb9
              7fab071
              9020864
              99be63c
              e27c016
              f6ae637
              9ab43ff
              fed8a1f
              796975b
              2df59d5
              File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -18,13 +18,22 @@ | |
| package org.apache.cassandra.db; | ||
| 
     | 
||
| import java.nio.ByteBuffer; | ||
| import java.util.Collection; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.Set; | ||
| import java.util.concurrent.TimeUnit; | ||
| 
     | 
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.base.Preconditions; | ||
| 
     | 
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| 
     | 
||
| import org.apache.cassandra.db.filter.DataLimits; | ||
| import org.apache.cassandra.index.Index; | ||
| import org.apache.cassandra.io.sstable.format.SSTableReader; | ||
| import org.apache.cassandra.replication.ShortMutationId; | ||
| import org.apache.cassandra.schema.TableMetadata; | ||
| import org.apache.cassandra.utils.MonotonicClock; | ||
| import org.apache.cassandra.utils.concurrent.OpOrder; | ||
| 
        
          
        
         | 
    @@ -33,6 +42,8 @@ | |
| 
     | 
||
| public class ReadExecutionController implements AutoCloseable | ||
| { | ||
| private static final Logger logger = LoggerFactory.getLogger(ReadExecutionController.class); | ||
| 
     | 
||
| private static final long NO_SAMPLING = Long.MIN_VALUE; | ||
| 
     | 
||
| // For every reads | ||
| 
        
          
        
         | 
    @@ -50,6 +61,13 @@ public class ReadExecutionController implements AutoCloseable | |
| private final RepairedDataInfo repairedDataInfo; | ||
| private long oldestUnrepairedTombstone = Long.MAX_VALUE; | ||
| 
     | 
||
| /* | ||
| * Track bulk transfers involved in the read, so we can do read reconciliation. | ||
| * These come from the ViewFragment, not the SSTable read path, so bloom filters + short-circuiting SSTable scans | ||
| * will still include the total set of relevant bulk transfers. | ||
| */ | ||
| private Set<ShortMutationId> activationIds = null; | ||
| 
     | 
||
| ReadExecutionController(ReadCommand command, | ||
| OpOrder.Group baseOp, | ||
| TableMetadata baseMetadata, | ||
| 
          
            
          
           | 
    @@ -243,4 +261,20 @@ private void addSample() | |
| if (cfs != null) | ||
| cfs.metric.topLocalReadQueryTime.addSample(cql, timeMicros); | ||
| } | ||
| 
     | 
||
| public void addActivationIds(ColumnFamilyStore.ViewFragment view) | ||
| { | ||
| activationIds = new HashSet<>(); | ||
                
       | 
||
| for (SSTableReader sstable : view.sstables) | ||
| 
         There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we should also validate that the table is part of a tracked keyspace  | 
||
| { | ||
| Collection<? extends ShortMutationId> ids = sstable.getCoordinatorLogOffsets().transfers(); | ||
| logger.trace("Adding transfer IDs from SSTable {} {}", sstable, ids); | ||
| activationIds.addAll(ids); | ||
| } | ||
| } | ||
| 
     | 
||
| public Iterator<ShortMutationId> getActivationIds() | ||
| { | ||
| return activationIds == null ? null : activationIds.iterator(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -44,6 +44,7 @@ | |
| import org.apache.cassandra.io.sstable.format.SSTableFormat.Components; | ||
| import org.apache.cassandra.io.sstable.format.SSTableReader; | ||
| import org.apache.cassandra.io.util.File; | ||
| import org.apache.cassandra.replication.MutationTrackingService; | ||
| import org.apache.cassandra.schema.TableMetadata; | ||
| import org.apache.cassandra.service.ActiveRepairService; | ||
| import org.apache.cassandra.service.StorageService; | ||
| 
          
            
          
           | 
    @@ -80,11 +81,8 @@ synchronized List<String> importNewSSTables(Options options) | |
| UUID importID = UUID.randomUUID(); | ||
| logger.info("[{}] Loading new SSTables for {}/{}: {}", importID, cfs.getKeyspaceName(), cfs.getTableName(), options); | ||
| 
     | 
||
| // This will be supported in the future | ||
| TableMetadata metadata = cfs.metadata(); | ||
| if (metadata.replicationType() != null && metadata.replicationType().isTracked()) | ||
| throw new IllegalStateException("Can't import into tables with mutation tracking enabled"); | ||
| 
     | 
||
| boolean isTracked = metadata.replicationType().isTracked(); | ||
| List<Pair<Directories.SSTableLister, String>> listers = getSSTableListers(options.srcPaths); | ||
| 
     | 
||
| Set<Descriptor> currentDescriptors = new HashSet<>(); | ||
| 
          
            
          
           | 
    @@ -183,7 +181,14 @@ synchronized List<String> importNewSSTables(Options options) | |
| Descriptor newDescriptor = cfs.getUniqueDescriptorFor(entry.getKey(), targetDir); | ||
| maybeMutateMetadata(entry.getKey(), options); | ||
| movedSSTables.add(new MovedSSTable(newDescriptor, entry.getKey(), entry.getValue())); | ||
| SSTableReader sstable = SSTableReader.moveAndOpenSSTable(cfs, entry.getKey(), newDescriptor, entry.getValue(), options.copyData); | ||
| SSTableReader sstable; | ||
| if (isTracked) | ||
| sstable = SSTableReader.open(cfs, oldDescriptor, metadata.ref); | ||
| 
         There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we stream even to the local node, we need to cleanup the source sstables. I think these are left on disk. We should also have a comment here explaining why we're not moving the sstable. I'd consider separating these 2 flows into separate methods or separate AbstractSSTableImport implementations, or something to that effect. The tracked and untracked flows have some fairly significant differences. I don't have super strong opinions about that, but the multiple uses of the isTracked flag throughout this fairly long method seems a bit brittle and makes it harder to read (imo)  | 
||
| else | ||
| { | ||
| // Don't move tracked SSTables, since that will move them to the live set on bounce | ||
| sstable = SSTableReader.moveAndOpenSSTable(cfs, oldDescriptor, newDescriptor, entry.getValue(), options.copyData); | ||
| } | ||
| newSSTablesPerDirectory.add(sstable); | ||
| } | ||
| catch (Throwable t) | ||
| 
          
            
          
           | 
    @@ -233,7 +238,13 @@ synchronized List<String> importNewSSTables(Options options) | |
| if (!cfs.indexManager.validateSSTableAttachedIndexes(newSSTables, false, options.validateIndexChecksum)) | ||
| cfs.indexManager.buildSSTableAttachedIndexesBlocking(newSSTables); | ||
| 
     | 
||
| cfs.getTracker().addSSTables(newSSTables); | ||
| if (isTracked) | ||
| { | ||
| TrackedBulkTransfer.execute(cfs.keyspace.getName(), newSSTables); | ||
| } | ||
| else | ||
| cfs.getTracker().addSSTables(newSSTables); | ||
| 
     | 
||
| for (SSTableReader reader : newSSTables) | ||
| { | ||
| if (options.invalidateCaches && cfs.isRowCacheEnabled()) | ||
| 
        
          
        
         | 
    @@ -250,6 +261,17 @@ synchronized List<String> importNewSSTables(Options options) | |
| return failedDirectories; | ||
| } | ||
| 
     | 
||
| /** | ||
| * TODO: Support user-defined consistency level for import, for import with replicas down | ||
| */ | ||
| private static class TrackedBulkTransfer | ||
| { | ||
| private static void execute(String keyspace, Set<SSTableReader> sstables) | ||
| { | ||
| MutationTrackingService.instance.executeTransfers(keyspace, sstables, ConsistencyLevel.ALL); | ||
| } | ||
| } | ||
| 
     | 
||
| /** | ||
| * Check the state of this node and throws an {@link InterruptedException} if it is currently draining | ||
| * | ||
| 
          
            
          
           | 
    ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -402,4 +402,4 @@ public boolean apply(T t) | |
| } | ||
| }; | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -46,19 +46,23 @@ | |
| import org.apache.cassandra.dht.Range; | ||
| import org.apache.cassandra.dht.Token; | ||
| import org.apache.cassandra.exceptions.UnknownColumnException; | ||
| import org.apache.cassandra.io.sstable.Descriptor; | ||
| import org.apache.cassandra.io.sstable.RangeAwareSSTableWriter; | ||
| import org.apache.cassandra.io.sstable.SSTableMultiWriter; | ||
| import org.apache.cassandra.io.sstable.SSTableSimpleIterator; | ||
| import org.apache.cassandra.io.sstable.SSTableTxnSingleStreamWriter; | ||
| import org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter; | ||
| import org.apache.cassandra.io.sstable.format.SSTableFormat; | ||
| import org.apache.cassandra.io.sstable.format.SSTableReader; | ||
| import org.apache.cassandra.io.sstable.format.Version; | ||
| import org.apache.cassandra.io.util.DataInputPlus; | ||
| import org.apache.cassandra.io.util.File; | ||
| import org.apache.cassandra.io.util.TrackedDataInputPlus; | ||
| import org.apache.cassandra.metrics.StorageMetrics; | ||
| import org.apache.cassandra.replication.ImmutableCoordinatorLogOffsets; | ||
| import org.apache.cassandra.schema.TableId; | ||
| import org.apache.cassandra.schema.TableMetadata; | ||
| import org.apache.cassandra.service.ActiveRepairService; | ||
| import org.apache.cassandra.service.StorageService; | ||
| import org.apache.cassandra.streaming.ProgressInfo; | ||
| import org.apache.cassandra.streaming.StreamReceivedOutOfTokenRangeException; | ||
| 
          
            
          
           | 
    @@ -180,15 +184,29 @@ protected SerializationHeader getHeader(TableMetadata metadata) throws UnknownCo | |
| } | ||
| protected SSTableTxnSingleStreamWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt, TimeUUID pendingRepair, ImmutableCoordinatorLogOffsets coordinatorLogOffsets, SSTableFormat<?, ?> format) throws IOException | ||
| { | ||
| boolean isTracked = cfs.metadata().replicationType().isTracked(); | ||
| 
     | 
||
| Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); | ||
| if (localDir == null) | ||
| throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); | ||
| 
     | 
||
| StreamReceiver streamReceiver = session.getAggregator(tableId); | ||
| Preconditions.checkState(streamReceiver instanceof CassandraStreamReceiver); | ||
| ILifecycleTransaction txn = createTxn(); | ||
| RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata())); | ||
| return new SSTableTxnSingleStreamWriter(txn, writer); | ||
| if (isTracked) | ||
                
       | 
||
| { | ||
| File location = cfs.getDirectories().getPendingLocationForDisk(localDir, session.planId()); | ||
| Descriptor desc = cfs.newSSTableDescriptor(location, format); | ||
| SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(desc, estimatedKeys, ActiveRepairService.UNREPAIRED_SSTABLE, ActiveRepairService.NO_PENDING_REPAIR, | ||
| coordinatorLogOffsets, cfs.metadata, null, sstableLevel, getHeader(cfs.metadata()), | ||
| cfs.indexManager.listIndexGroups(), txn, cfs); | ||
| return new SSTableTxnSingleStreamWriter(txn, writer); | ||
| } | ||
| else | ||
| { | ||
| RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata())); | ||
| return new SSTableTxnSingleStreamWriter(txn, writer); | ||
| } | ||
| } | ||
| 
     | 
||
| private ILifecycleTransaction createTxn() | ||
| 
          
            
          
           | 
    ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -52,6 +52,8 @@ | |
| import org.apache.cassandra.service.accord.AccordTopology; | ||
| import org.apache.cassandra.service.accord.IAccordService; | ||
| import org.apache.cassandra.service.accord.TimeOnlyRequestBookkeeping.LatencyRequestBookkeeping; | ||
| import org.apache.cassandra.replication.MutationTrackingService; | ||
| import org.apache.cassandra.replication.PendingLocalTransfer; | ||
| import org.apache.cassandra.streaming.IncomingStream; | ||
| import org.apache.cassandra.streaming.StreamReceiver; | ||
| import org.apache.cassandra.streaming.StreamSession; | ||
| 
          
            
          
           | 
    @@ -256,6 +258,15 @@ public void finished() | |
| 
     | 
||
| // add sstables (this will build non-SSTable-attached secondary indexes too, see CASSANDRA-10130) | ||
| logger.debug("[Stream #{}] Received {} sstables from {} ({})", session.planId(), readers.size(), session.peer, readers); | ||
| 
     | 
||
| if (cfs.metadata().replicationType().isTracked()) | ||
                
       | 
||
| { | ||
| // Don't mark as live until activated by the stream coordinator | ||
| PendingLocalTransfer transfer = new PendingLocalTransfer(cfs.metadata().id, session.planId(), sstables); | ||
| MutationTrackingService.instance.received(transfer); | ||
| return; | ||
| } | ||
| 
     | 
||
| cfs.addSSTables(readers); | ||
| 
     | 
||
| //invalidate row and counter cache | ||
| 
          
            
          
           | 
    ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
discussed elsewhere, but we should only be adding activation ids that intersect with the key/range of the read