You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/06/18 10:18:26 UTC

[GitHub] [cassandra] jasonstack opened a new pull request #642: Cassandra 15861 lock trunk

jasonstack opened a new pull request #642:
URL: https://github.com/apache/cassandra/pull/642


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469593117



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       There are other places that write `indexSummary` without synchronization as well, namely `SSTableReader#load()` and `openForBatch()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470086173



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       +1, that would be a big improvement




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470073171



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I've had a chance to talk it over w/ @belliottsmith and think about it, and tl;dr the changes here should be safe. The big thing I was missing is that the `SSTableReader` itself should already be published safely between threads, and if it is, the `indexSummary` reference should be fine (i.e. not null, which is the only danger, because it doesn't actually change after creation given we clone).
   
   However, it might be helpful to briefly document this in `cloneWithNewSummarySamplingLevel()` so future readers might avoid some confusion. Better yet, we could finally just make these fields final in `SSTableReader`, avoiding all this confusion in the first place. I'm guessing the reason they aren't is that we added some layers of initialization over time, but deferring reader creation until we're ready seems like it could be achieved with a builder (or even without one).
   
   CC @jasonstack 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473142375



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -701,16 +633,24 @@ protected SSTableReader(final Descriptor desc,
                             long maxDataAge,
                             StatsMetadata sstableMetadata,
                             OpenReason openReason,
-                            SerializationHeader header)
+                            SerializationHeader header,
+                            IndexSummary summary,
+                            FileHandle dfile,
+                            FileHandle ifile,
+                            IFilter bf)
     {
         super(desc, components, metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
         this.sstableMetadata = sstableMetadata;
         this.header = header;
+        this.indexSummary = summary;
         this.maxDataAge = maxDataAge;
         this.openReason = openReason;
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), desc.version, header);
         tidy = new InstanceTidier(descriptor, metadata.id);
         selfRef = new Ref<>(this, tidy);
+        this.dfile = dfile;
+        this.ifile = ifile;
+        this.bf = bf;

Review comment:
       OCD: Perhaps assign these together w/ the summary?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483976245



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       > org.apache.cassandra.streaming.async.NettyStreamingMessageSender#sendControlMessage
   
   This is for `KeepAliveMessage`..
   
   For `OutgoingStreamMessage`, it must be `AsyncStreamingOutputPlus `




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473120914



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -626,42 +575,25 @@ public void run()
      * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
      */
     public static SSTableReader internalOpen(Descriptor desc,
-                                      Set<Component> components,
-                                      TableMetadataRef metadata,
-                                      FileHandle ifile,
-                                      FileHandle dfile,
-                                      IndexSummary isummary,
-                                      IFilter bf,
-                                      long maxDataAge,
-                                      StatsMetadata sstableMetadata,
-                                      OpenReason openReason,
-                                      SerializationHeader header)
-    {
-        assert desc != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
-
-        SSTableReader reader = internalOpen(desc, components, metadata, maxDataAge, sstableMetadata, openReason, header);
-
-        reader.bf = bf;
-        reader.ifile = ifile;
-        reader.dfile = dfile;
-        reader.indexSummary = isummary;
-        reader.setup(true);
-
-        return reader;
-    }
-
-
-    private static SSTableReader internalOpen(final Descriptor descriptor,
-                                              Set<Component> components,
-                                              TableMetadataRef metadata,
-                                              Long maxDataAge,
-                                              StatsMetadata sstableMetadata,
-                                              OpenReason openReason,
-                                              SerializationHeader header)
-    {
-        Factory readerFactory = descriptor.getFormat().getReaderFactory();
-
-        return readerFactory.open(descriptor, components, metadata, maxDataAge, sstableMetadata, openReason, header);
+                                             Set<Component> components,
+                                             TableMetadataRef metadata,
+                                             FileHandle ifile,
+                                             FileHandle dfile,
+                                             IFilter bf,
+                                             long maxDataAge,
+                                             StatsMetadata sstableMetadata,
+                                             OpenReason openReason,
+                                             SerializationHeader header,
+                                             IndexSummary summary)
+    {
+        assert desc != null && ifile != null && dfile != null && summary != null && bf != null && sstableMetadata != null;
+
+        BuilderForWriter builder = new BuilderForWriter(desc, metadata, maxDataAge);
+        builder.bf = bf;
+        builder.ifile = ifile;
+        builder.dfile = dfile;
+        builder.summary = summary;

Review comment:
       nit: We could make a minor clean up here and have "chainable" methods for setting the bloom filter if you think that reads better.
   
   ex.
   ```
   BuilderForWriter builder =
       new BuilderForWriter(desc, metadata, maxDataAge).bloomFilter(bf)
                                                       .indexFile(ifile)
                                                       .dataFile(dfile)
                                                       .indexSummary(summary)
                                                       .build(components, sstableMetadata, openReason, header);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470015470



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       There _is_ a volatile write to `setup` in `InstanceTidier#setup()` after `ifile`, `indexSummary`, et al. are set on the `SSTableReader`, but I'm not sure that helps us reason about the visibility of `indexSummary` for the thread that enters `cloneWithNewSummarySamplingLevel()`?
   
   Unless there's some consequence I'm missing around making `global` final and assigning it at `InstanceTidier` (which would at least get the static analysis tools off our backs), I'd really like to make our synchronization policy around the non-final fields of the reader explicit (i.e. around `tidy.global`) rather than relying on more subtle reasoning around a volatile.
   
   In terms of how that all affects this patch, the fact that `IndexSummary` is immutable should make it possible for us to synchronize only to get the `IndexSummary` reference, rather than having to wait for `buildSummaryAtLevel()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481913923



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       @maedhroz @dcapwell but then, suddenly, we are perfecty happy to cast that into `AsyncStreamingOutputPlus` without any checks and that write method also takes DataOutputStreamPlus ... Same for uncompressed / normal writer.
    https://github.com/apache/cassandra/blob/7542fd81abb445e10fbcc67929d742a9849b7129/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java#L61




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r482606854



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       that seems problematic as `org.apache.cassandra.streaming.async.NettyStreamingMessageSender#sendControlMessage` calls this method (transitively) and passes in a `DataOutputBufferFixed`, which is not a `AsyncStreamingOutputPlus`.
   
   I would need to look closer to see how they relate, but it does seem like this logic isn't safe.
   
   Now, given the fact that this patch doesn't change this logic and the logic is already there; I am fine ignoring this fact in this JIRA, though that should be looked at closer as this seems like an edge case where `header.compressionInfo != null` might fail




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483977041



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?
-                     new CassandraStreamWriter(sstable, header.sections, session) :
-                     new CassandraCompressedStreamWriter(sstable, header.sections,
-                                                         header.compressionInfo, session);
+                                           new CassandraStreamWriter(sstable, header.sections, session) :
+                                           new CassandraCompressedStreamWriter(sstable, header.sections, header.compressionInfo, session);

Review comment:
       good idea




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459537811



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,14 +122,23 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
         AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469588637



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       @bdeggleston @jasonstack I [mentioned](https://issues.apache.org/jira/browse/CASSANDRA-15861?focusedCommentId=17170484&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17170484) this briefly in the Jira, but with `indexSummary` not `final` or `volatile` in `SSTableReader`, do we have a visibility issue? The thread that loads/builds the summary needs to safely publish its changes for the thread that builds new summaries here.
   
   Simply making `indexSummary` `volatile` might work. Most of its accesses are themselves part of the summary redistribution task itself or during compaction, so we probably don't care about that overhead. `SSTableReader#getIndexScanPosition()`, on the other hand, does look like it's used at least during some range reads.
   
   We could go back to the old scope for the `synchronized` block, leaving `indexSummary` non-volatile. That would at least ensure visibility between the `cloneX()` methods. Even then, there are other things that call `SSTableReader#internalOpen()`, like `BigTableWriter.openEarly()` and `openFinal()`, that don't initialize `indexSummary` while holding the global lock. (Perhaps we could just synchronize to get the `IndexSummary` reference, rather than having to wait for `buildSummaryAtLevel()`, though.)
   
   Is there a bug here independent of this patch? i.e. Do we actually need to make sure the block that includes setting `indexSummary` in `SSTableReader#internalOpen()` is performed in a `synchronized (tidy.global)` block? We can't synchronize on `tidy`, given we create a new one for a new reader, but we should be able to sync on the right `GlobalTidy` even before set call `setup()`...or we could make `global` final and initialize it at `InstanceTidier` construction.
   
   I hope I'm just missing something...
   
   CC @belliottsmith 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473136983



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
##########
@@ -103,9 +104,11 @@ public SSTableWriter open(Descriptor descriptor,
     static class ReaderFactory extends SSTableReader.Factory
     {
         @Override
-        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata,
+                                  Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason,
+                                  SerializationHeader header, IndexSummary summary, FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       nit: For now I guess we should at least put the arguments on separate lines.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r461119147



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -55,55 +56,79 @@
     private final long estimatedKeys;
     private final List<SSTableReader.PartitionPositionBounds> sections;
     private final String filename;
-    private final CassandraStreamHeader header;
-    private final boolean keepSSTableLevel;
-    private final ComponentManifest manifest;
-
     private final boolean shouldStreamEntireSSTable;
+    private final StreamOperation operation;
+    private final CassandraStreamHeader header;
 
     public CassandraOutgoingFile(StreamOperation operation, Ref<SSTableReader> ref,
                                  List<SSTableReader.PartitionPositionBounds> sections, List<Range<Token>> normalizedRanges,
                                  long estimatedKeys)
     {
         Preconditions.checkNotNull(ref.get());
         Range.assertNormalized(normalizedRanges);
+        this.operation = operation;
         this.ref = ref;
         this.estimatedKeys = estimatedKeys;
         this.sections = sections;
-        this.filename = ref.get().getFilename();
-        this.manifest = getComponentManifest(ref.get());
-        this.shouldStreamEntireSSTable = computeShouldStreamEntireSSTables();
 
         SSTableReader sstable = ref.get();
-        keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD;
-        this.header =
-            CassandraStreamHeader.builder()
-                                 .withSSTableFormat(sstable.descriptor.formatType)
-                                 .withSSTableVersion(sstable.descriptor.version)
-                                 .withSSTableLevel(keepSSTableLevel ? sstable.getSSTableLevel() : 0)
-                                 .withEstimatedKeys(estimatedKeys)
-                                 .withSections(sections)
-                                 .withCompressionMetadata(sstable.compression ? sstable.getCompressionMetadata() : null)
-                                 .withSerializationHeader(sstable.header.toComponent())
-                                 .isEntireSSTable(shouldStreamEntireSSTable)
-                                 .withComponentManifest(manifest)
-                                 .withFirstKey(sstable.first)
-                                 .withTableId(sstable.metadata().id)
-                                 .build();
+
+        this.filename = sstable.getFilename();
+        this.shouldStreamEntireSSTable = computeShouldStreamEntireSSTables();
+        ComponentManifest manifest = getComponentManifest(sstable, false);
+        this.header = makeHeader(sstable, operation, sections, estimatedKeys, shouldStreamEntireSSTable, manifest);
+    }
+
+    private static CassandraStreamHeader makeHeader(SSTableReader sstable,
+                                                    StreamOperation operation,
+                                                    List<SSTableReader.PartitionPositionBounds> sections,
+                                                    long estimatedKeys,
+                                                    boolean shouldStreamEntireSSTable,
+                                                    ComponentManifest manifest)
+    {
+        boolean keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD;
+
+        return CassandraStreamHeader.builder()
+                                    .withSSTableFormat(sstable.descriptor.formatType)
+                                    .withSSTableVersion(sstable.descriptor.version)
+                                    .withSSTableLevel(keepSSTableLevel ? sstable.getSSTableLevel() : 0)
+                                    .withEstimatedKeys(estimatedKeys)
+                                    .withSections(sections)
+                                    .withCompressionMetadata(sstable.compression ? sstable.getCompressionMetadata() : null)
+                                    .withSerializationHeader(sstable.header.toComponent())
+                                    .isEntireSSTable(shouldStreamEntireSSTable)
+                                    .withComponentManifest(manifest)
+                                    .withFirstKey(sstable.first)
+                                    .withTableId(sstable.metadata().id)
+                                    .build();
     }
 
     @VisibleForTesting
-    public static ComponentManifest getComponentManifest(SSTableReader sstable)
+    public static ComponentManifest getComponentManifest(SSTableReader sstable, boolean createHardLink)

Review comment:
       @jasonstack What if we just moved `getComponentManifest()` to `ComponentManifest` itself as a static factory, maybe `ComponentManifest.create()`, and centralized all manifest creation there? (Might have to move `STREAM_COMPONENTS` there as well?)

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2078,25 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate Stats Metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     *
+     * @param transform function to mutate stats metadata
+     */
+    public void mutateAndReloadStats(UnaryOperator<StatsMetadata> transform) throws IOException
+    {
+        tidy.global.writeLock.lock();
+        try
+        {
+            descriptor.getMetadataSerializer().mutate(descriptor, transform);
+            reloadSSTableMetadata();
+        }
+        finally
+        {
+            tidy.global.writeLock.unlock();
+        }

Review comment:
       @jasonstack Let's move this block to `runWithWriteLock()`. It might help future readers looking at usages of `runWithWriteLock()` trace back to the stats writing code itself, rather than having to search on usages of the write lock itself.

##########
File path: test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java
##########
@@ -127,7 +127,7 @@ public void test() throws Exception
 
         int totalNumberOfFiles = session.transfers.get(store.metadata.id).getTotalNumberOfFiles();
 
-        assertEquals(CassandraOutgoingFile.getComponentManifest(sstable).components().size(), totalNumberOfFiles);
+        assertEquals(CassandraOutgoingFile.getComponentManifest(sstable, false).components().size(), totalNumberOfFiles);

Review comment:
       @jasonstack Would it be useful to have a simple test that ensures hard links get cleaned up when streaming completes? (perhaps an additional assertion in one of the other tests...)

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +179,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> getComponentManifest(sstable, true));

Review comment:
       @jasonstack If we only hold the read lock for manifest creation, and we _don't_ create a hard link for the index summary component, what happens if, after we create the manifest, we start streaming, and then an index summary redistribution finishes and tries to save? It seems like creating a hard link for the index summary would work here, even if we don't do an atomic replace, given manifest creation would have to block if a summary saving is in progress anyway.

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       @jasonstack Out of curiosity, I disabled all the read/write lock acquisition in this patch, and the tests here all seem to pass pretty consistently still. I think what we might need to do here is take the concept here, and most of the code, and test w/ two threads, one streaming at the `CassandraOutgoingFile#write()` level, where our read locking actually occurs, and the other guaranteed to make a mutation to the stats or index summary between initial manifest creation and streaming. (i.e. Something that would clearly fail before this patch.)
   
   I think the other thing that made this more apparent to me after looking a second time was the fact that we actually call `runWithReadLock()` explicitly in `testBlockReadingAndWritingOverWire()`. We should be testing the locking mechanisms in a more black box fashion, not replacing part of that mechanism in the test.
   
   WDYT?

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       @jasonstack Out of curiosity, I disabled all the read/write lock acquisition in this patch, and the tests here all seem to pass pretty consistently still. I think what we might need to do is take the concept here, and most of the code, and test w/ two threads, one streaming at the `CassandraOutgoingFile#write()` level, where our read locking actually occurs, and the other guaranteed to make a mutation to the stats or index summary between initial manifest creation and streaming. (i.e. Something that would clearly fail before this patch.)
   
   I think the other thing that made this more apparent to me after looking a second time was the fact that we actually call `runWithReadLock()` explicitly in `testBlockReadingAndWritingOverWire()`. We should be testing the locking mechanisms in a more black box fashion, not replacing part of that mechanism in the test.
   
   WDYT?

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       @jasonstack Out of curiosity, I disabled all the read/write lock acquisition in this patch, and the tests here all seem to pass pretty consistently still. I think what we might need to do is take the concept here, and most of the code, and test w/ two threads, one streaming at the `CassandraOutgoingFile#write()` level, where our read locking actually occurs, and the other guaranteed to make a mutation to the stats or index summary between initial manifest creation and streaming. (i.e. Something that would clearly fail before this patch.)
   
   I think the other thing that made this more apparent to me after looking a second time was the fact that we actually call `runWithReadLock()` explicitly in `testBlockReadingAndWritingOverWire()`. We should be testing the locking mechanism in a more black box fashion, not replacing part of that mechanism in the test itself.
   
   WDYT?

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       @jasonstack Out of curiosity, I disabled all the read/write lock acquisition in this patch, and the tests here all seem to pass pretty consistently still. I think what we might need to do is take the concept here, and most of the code, and test w/ two threads, one streaming at the `CassandraOutgoingFile#write()` level, where our read locking actually occurs, and the other guaranteed to make a mutation to the stats or index summary between initial manifest creation and streaming. (i.e. Something that would clearly fail before this patch.)
   
   I think the other thing that made this more apparent to me after looking a second time was the fact that we actually call `runWithReadLock()` explicitly in `testBlockReadingAndWritingOverWire()`. We should be testing the locking mechanism in a more black box fashion, not replacing part of that mechanism in the test itself.
   
   WDYT?
   
   (I don't know if we should literally move the tests to `CassandraOutgoingFileTest`, but `CassandraEntireSSTableStreamWriter` is clearly not where we added the new logic.)

##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -18,28 +18,85 @@
 
 package org.apache.cassandra.db.streaming;
 
+import java.io.File;
 import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileUtils;
 
 public final class ComponentManifest implements Iterable<Component>
 {
+    private static final List<Component> STREAM_COMPONENTS = ImmutableList.of(Component.DATA, Component.PRIMARY_INDEX, Component.STATS,
+                                                                             Component.COMPRESSION_INFO, Component.FILTER, Component.SUMMARY,
+                                                                             Component.DIGEST, Component.CRC);
+
+    private static final Set<Component> MUTABLED_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);

Review comment:
       nit: `MUTABLED_COMPONENTS` -> `MUTABLE_COMPONENTS`

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -112,7 +111,7 @@ public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, ComponentManifest.create(sstable, false));
 
         EmbeddedChannel channel = new EmbeddedChannel();

Review comment:
       @jasonstack Should we still close this channel? (It looks like we more or less reverted this back to what's in trunk, which is fine, but wanted to make sure we didn't throw the baby out with the bath water...)

##########
File path: src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
##########
@@ -1200,8 +1200,7 @@ public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt,
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
-                sstable.reloadSSTableMetadata();
+                sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(repairedAt, pendingRepair, isTransient));

Review comment:
       @jasonstack Ah, just a reminder to put this back if possible before we commit.

##########
File path: src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
##########
@@ -1200,8 +1200,7 @@ public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt,
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
-                sstable.reloadSSTableMetadata();
+                sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(repairedAt, pendingRepair, isTransient));

Review comment:
       @jasonstack Ah, just a reminder to put this back if possible before we commit, given we went the hard-link route.

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")
+    public void testStreamWithIndexSummaryRedistributionDelaySavingSummary() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(() -> {
+            // wait until new index summary is partially written
+            latch.await(1, TimeUnit.MINUTES);
+            return null;
+        }, this::indexSummaryRedistribution);
+    }
+
+    // used by byteman
+    private static void countDown()
+    {
+        latch.countDown();
+    }

Review comment:
       formatting: missing newline?

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")

Review comment:
       formatting: horizontal alignment w/ `name`?

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),

Review comment:
       We don't actually need to load/create these other two column families for this test, right?

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")
+    public void testStreamWithIndexSummaryRedistributionDelaySavingSummary() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(() -> {
+            // wait until new index summary is partially written
+            latch.await(1, TimeUnit.MINUTES);
+            return null;
+        }, this::indexSummaryRedistribution);
+    }
+
+    // used by byteman
+    private static void countDown()
+    {
+        latch.countDown();
+    }
+    private void testStreamWithConcurrentComponentMutation(Callable<?> runBeforeStreaming, Callable<?> mutation) throws Exception
+    {
+        ByteBuf serializedFile = Unpooled.buffer(8192);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        Collection<OutgoingStream> outgoingStreams = store.getStreamManager().createOutgoingStreams(session, rangesAtEndpoint, NO_PENDING_REPAIR, PreviewKind.NONE);
+        CassandraOutgoingFile outgoingFile = (CassandraOutgoingFile) Iterables.getOnlyElement(outgoingStreams);
+
+        Future<?> streaming = executeAsync(() -> {
+            runBeforeStreaming.call();
+
+            try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(createMockNettyChannel(serializedFile)))
+            {
+                outgoingFile.write(session, out, MessagingService.current_version);
+                assertTrue(sstable.descriptor.getTemporaryFiles().isEmpty());
+            }
+            return null;
+        });
+
+        Future<?> concurrentMutations = executeAsync(mutation);
+
+        streaming.get(3, TimeUnit.MINUTES);
+        concurrentMutations.get(3, TimeUnit.MINUTES);
+
+        session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104));
+        StreamMessageHeader messageHeader = new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null);
+
+        try (DataInputBuffer in = new DataInputBuffer(serializedFile.nioBuffer(), false))
+        {
+            CassandraStreamHeader header = CassandraStreamHeader.serializer.deserialize(in, MessagingService.current_version);
+            CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(messageHeader, header, session);
+            SSTableReader streamedSSTable = Iterables.getOnlyElement(reader.read(in).finished());
+
+            SSTableUtils.assertContentEquals(sstable, streamedSSTable);
+        }
+    }
+
+    private boolean indexSummaryRedistribution() throws IOException
+    {
+        long nonRedistributingOffHeapSize = 0;
+        long memoryPoolBytes = 1024 * 1024;
+
+        // rewrite index summary file with new min/max index interval
+        TableMetadata origin = store.metadata();
+        MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstable, OperationType.INDEX_SUMMARY))
+        {
+            IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                     nonRedistributingOffHeapSize,
+                                                                                     memoryPoolBytes));
+        }
+
+        // reset min/max index interval
+        MigrationManager.announceTableUpdate(origin, true);
+        return true;
+    }
+
+    private Future<?> executeAsync(Callable<?> mutation)

Review comment:
       nit: This isn't always a mutation.

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")

Review comment:
       @jasonstack Awesome. I think this is going to work in most environments. (It certainly works locally for me.) What if, instead of relying on a 5 second wait, we just wait on a separate latch, sort of the opposite of what we do with `countDown()` immediately before? Then we'll suspend the summary saving from finishing while the write continues. When the write is done, we can have it count down on that second latch and free the other thread to finish the save. WDYT?
   
   (I know we've both been bitten by slow testing environments messing with our timing assumptions in the past.)

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {

Review comment:
       @jasonstack Crazy idea: What if we just did the mutation as the `runBeforeStreaming` callable here? We could at least avoid the hard-coded sleep (although the `mutation` callable might need a more generalized name, like `runConcurrentWithStreaming` or something).

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2081,21 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate Stats Metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     *
+     * @param description on changed attributes
+     * @param transform function to mutate stats metadata
+     */
+    public void mutateAndReloadStats(String description, UnaryOperator<StatsMetadata> transform) throws IOException

Review comment:
       So I guess there's one other thing you could do that would again simplify the tracing stuff we added back, and that's breaking `mutateAndReloadStats()` back into a few different methods. (i.e. One for changing repairedAt time, one for level changes, etc.) It might at least make piecing together the actual trace message easier. The `runWithWriteLock()` wrapping isn't really that much boilerplate to duplicate among them for the sake of clarity.

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> ComponentManifest.create(sstable, true));

Review comment:
       I suppose we could make `ComponentManifest` `AutoCloseable`. (I guess it wouldn't be necessary to close in all cases though...)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471579592



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       @jasonstack I took a look and dropped a couple nits. Overall, it looks pretty straightforward, a largely organizational change that isolates building logic from the reader itself. I _do_ think that we should make `dfile` / `ifile` / `bf` final while we're here and have the opportunity. `first` and `last` seem a little more complicated, given the way the writer uses them during append, so no big deal if those stay mutable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459348675



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1199,7 +1219,15 @@ else if (samplingLevel < indexSummary.getSamplingLevel())
             }
 
             // Always save the resampled index
-            saveSummary(newSummary);
+            tidy.global.writeLock.lock();
+            try
+            {
+                saveSummary(newSummary);

Review comment:
       yeah, this is the only one. I think the other index-summary modifications are used before being part of the node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458438151



##########
File path: src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
##########
@@ -1200,8 +1200,7 @@ public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt,
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
-                sstable.reloadSSTableMetadata();
+                sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(repairedAt, pendingRepair, isTransient));

Review comment:
       @jasonstack Do we lose trace recording if we call `mutate()` directly rather than `IMetadataSerializer#mutateRepairMetadata()`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459525236



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);

Review comment:
       good catch.. will remove it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470015470



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       There _is_ a volatile write to `setup` in `InstanceTidier#setup()` after `ifile`, `indexSummary`, et al. are set on the `SSTableReader`, but I'm not sure that helps us reason about the visibility of `indexSummary` for the thread that enters `cloneWithNewSummarySamplingLevel()`?
   
   Unless there's some consequence I'm missing around making `global` final and assigning it at `InstanceTidier` construction (which would at least get the static analysis tools off our backs), I'd really like to make our synchronization policy around the non-final fields of the reader explicit (i.e. around `tidy.global`) rather than relying on more subtle reasoning around a volatile.
   
   In terms of how that all affects this patch, the fact that `IndexSummary` is immutable should make it possible for us to synchronize only to get the `IndexSummary` reference, rather than having to wait for `buildSummaryAtLevel()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473655577



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2538,4 +2298,369 @@ public static void shutdownBlocking(long timeout, TimeUnit unit) throws Interrup
         ExecutorUtils.shutdownNowAndWait(timeout, unit, syncExecutor);
         resetTidying();
     }
+    
+    private static abstract class Builder
+    {
+        protected final Descriptor descriptor;
+        protected final TableMetadataRef metadataRef;
+        protected final TableMetadata metadata;
+        protected final Factory readerFactory;
+
+        public IndexSummary summary;
+        public DecoratedKey first;
+        public DecoratedKey last;
+        public IFilter bf;
+        public FileHandle ifile;
+        public FileHandle dfile;
+
+        public Builder(Descriptor descriptor, TableMetadataRef metadataRef)
+        {
+            this.descriptor = descriptor;
+            this.metadataRef = metadataRef;
+            this.metadata = metadataRef.get();
+            this.readerFactory = descriptor.getFormat().getReaderFactory();
+        }
+
+        /**
+         * Load index summary from Summary.db file if it exists.
+         *
+         * if loaded index summary has different index interval from current value stored in schema,
+         * then Summary.db file will be deleted and this returns false to rebuild summary.
+         */
+        void loadSummary()
+        {
+            File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+            if (!summariesFile.exists())
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+                return;
+            }
+
+            DataInputStream iStream = null;
+            try
+            {
+                iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+                summary = IndexSummary.serializer.deserialize(iStream,
+                                                              metadata.partitioner,
+                                                              metadata.params.minIndexInterval,
+                                                              metadata.params.maxIndexInterval);
+                first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+                last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            }
+            catch (IOException e)
+            {
+                if (summary != null)
+                    summary.close();
+                logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+                // corrupted; delete it and fall back to creating a new summary
+                FileUtils.closeQuietly(iStream);
+                // delete it and fall back to creating a new summary
+                FileUtils.deleteWithConfirm(summariesFile);
+            }
+            finally
+            {
+                FileUtils.closeQuietly(iStream);
+            }
+        }
+
+        /**
+         * Build index summary(and optionally bloom filter) by reading through Index.db file.
+         *
+         * @param recreateBloomFilter true if recreate bloom filter
+         * @param summaryLoaded true if index summary is already loaded and not need to build again
+         */
+        void buildSummary(boolean recreateBloomFilter,
+                          boolean summaryLoaded,
+                          Set<Component> components,
+                          StatsMetadata statsMetadata) throws IOException
+        {
+            if (!components.contains(Component.PRIMARY_INDEX))
+                return;
+
+            if (logger.isDebugEnabled())
+                logger.debug("Attempting to build summary for {}", descriptor);
+
+
+            // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+            try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+            {
+                long indexSize = primaryIndex.length();
+                long histogramCount = statsMetadata.estimatedPartitionSize.count();
+                long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                     ? histogramCount
+                                     : estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+                if (recreateBloomFilter)
+                    bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+                try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+                {
+                    long indexPosition;
+
+                    while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                    {
+                        ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                        RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                        DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                        if (first == null)
+                            first = decoratedKey;
+                        last = decoratedKey;
+
+                        if (recreateBloomFilter)
+                            bf.add(decoratedKey);
+
+                        // if summary was already read from disk we don't want to re-populate it using primary index
+                        if (!summaryLoaded)
+                        {
+                            summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
+                        }
+                    }
+
+                    if (!summaryLoaded)
+                        summary = summaryBuilder.build(metadata.partitioner);
+                }
+            }
+
+            first = getMinimalKey(first);
+            last = getMinimalKey(last);
+        }
+
+        /**
+         * Load bloom filter from Filter.db file.
+         *
+         * @throws IOException
+         */
+        IFilter loadBloomFilter() throws IOException
+        {
+            try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
+            {
+                return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat());
+            }
+        }
+
+        public abstract SSTableReader build(Set<Component> components, StatsMetadata statsMetadata, OpenReason reason, SerializationHeader header);

Review comment:
       because it uses the concrete type.. 

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2430,7 +2187,10 @@ public abstract SSTableReader open(final Descriptor descriptor,
                                            Long maxDataAge,
                                            StatsMetadata sstableMetadata,
                                            OpenReason openReason,
-                                           SerializationHeader header);
+                                           SerializationHeader header,
+                                           IndexSummary summary,
+                                           FileHandle dfile,
+                                           FileHandle ifile, IFilter bf);

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471222401



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       Does "resource" actually avoid the warning?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481854790



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(ComponentContext.class);
+
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);

Review comment:
       reverted to hashmap




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473137482



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
##########
@@ -53,9 +55,11 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(BigTableReader.class);
 
-    BigTableReader(Descriptor desc, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header)
+    BigTableReader(Descriptor desc, Set<Component> components, TableMetadataRef metadata, Long maxDataAge,
+                   StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header, IndexSummary summary,
+                   FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       nit: newlines for arguments




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483975723



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469469409



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       @jasonstack I think what we actually want is a `@SuppressWarnings("IOResourceOpenedButNotSafelyClosed")` on the line where we create the `RandomAccessFile`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481411958



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,
+                      boolean summaryLoaded,
+                      Set<Component> components,
+                      StatsMetadata statsMetadata) throws IOException
+    {
+        if (!components.contains(Component.PRIMARY_INDEX))
+            return;
+
+        if (logger.isDebugEnabled())
+            logger.debug("Attempting to build summary for {}", descriptor);
+
+
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+        {
+            long indexSize = primaryIndex.length();
+            long histogramCount = statsMetadata.estimatedPartitionSize.count();
+            long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                 ? histogramCount
+                                 : SSTable.estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+            if (recreateBloomFilter)
+                bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+            {
+                long indexPosition;
+
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                    RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                    DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                    if (first == null)
+                        first = decoratedKey;
+                    last = decoratedKey;
+
+                    if (recreateBloomFilter)
+                        bf.add(decoratedKey);
+
+                    // if summary was already read from disk we don't want to re-populate it using primary index
+                    if (!summaryLoaded)
+                    {
+                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
+                    }
+                }
+
+                if (!summaryLoaded)
+                    summary = summaryBuilder.build(metadata.partitioner);
+            }
+        }
+
+        first = SSTable.getMinimalKey(first);
+        last = SSTable.getMinimalKey(last);
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     *
+     * @throws IOException
+     */
+    IFilter loadBloomFilter() throws IOException
+    {
+        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
+        {
+            return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat());
+        }
+    }
+
+    public static class ForWriter extends SSTableReaderBuilder
+    {
+        public ForWriter(Descriptor descriptor,
+                         TableMetadataRef metadataRef,
+                         long maxDataAge,
+                         Set<Component> components,
+                         StatsMetadata statsMetadata,
+                         SSTableReader.OpenReason openReason,
+                         SerializationHeader header)
+        {
+            super(descriptor, metadataRef, maxDataAge, components, statsMetadata, openReason, header);
+        }
+
+        @Override
+        public SSTableReader build()
+        {
+            SSTableReader reader = readerFactory.open(this);
+
+            reader.setup(true);
+            return reader;
+        }
+    }
+
+    public static class ForBatch extends SSTableReaderBuilder
+    {
+        public ForBatch(Descriptor descriptor,
+                        TableMetadataRef metadataRef,
+                        Set<Component> components,
+                        StatsMetadata statsMetadata,
+                        SerializationHeader header)
+        {
+            super(descriptor, metadataRef, System.currentTimeMillis(), components, statsMetadata, SSTableReader.OpenReason.NORMAL, header);
+        }
+
+        @Override
+        public SSTableReader build()
+        {
+            String dataFilePath = descriptor.filenameFor(Component.DATA);
+            long fileLength = new File(dataFilePath).length();
+            logger.info("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength));
+
+            initSummary(dataFilePath, components, statsMetadata);
+
+            boolean compression = components.contains(Component.COMPRESSION_INFO);
+            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
+                    .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+                    .withChunkCache(ChunkCache.instance);
+                    FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)
+                                                                                                                .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
+                                                                                                                .withChunkCache(ChunkCache.instance))
+            {
+                long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+                DiskOptimizationStrategy optimizationStrategy = DatabaseDescriptor.getDiskOptimizationStrategy();
+                int dataBufferSize = optimizationStrategy.bufferSize(statsMetadata.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
+                int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / summary.size());
+                ifile = ibuilder.bufferSize(indexBufferSize).complete();
+                dfile = dbuilder.bufferSize(dataBufferSize).complete();
+                bf = FilterFactory.AlwaysPresent;
+
+                SSTableReader sstable = readerFactory.open(this);
+
+                sstable.first = first;

Review comment:
       nit: would have loved if the builders made first/last immutable.  At the moment the read logic is now controlled via builder, the only complication is `org.apache.cassandra.io.sstable.format.big.BigTableWriter#afterAppend`.
   
   this is unrelated to the zero copy work and locking, so fine to leave as is




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459634173



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -30,16 +35,31 @@
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.streaming.StreamManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static java.lang.Math.min;
 
 public final class ComponentManifest implements Iterable<Component>

Review comment:
       We could almost call this thing `ComponentStreamer` now :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459524831



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473679730



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1125,18 +839,47 @@ private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason,
                                                  metadata,
                                                  ifile != null ? ifile.sharedCopy() : null,
                                                  dfile.sharedCopy(),
-                                                 newSummary,
                                                  bf.sharedCopy(),
                                                  maxDataAge,
                                                  sstableMetadata,
                                                  reason,
-                                                 header);
+                                                 header,
+                                                 newSummary);

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack edited a comment on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack edited a comment on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-684340690


   > a slow write to SUMMARY (aka IndexSummaryRedistribution) will block updates to STATS (aka compaction) even though they do not logically conflict; 
   
   index summary redistribution and stats-mutation shouldn't happen together because both are running as part of compactions.
   
   > streaming is only blocked while snapshotting.
   
   streaming is also blocked by "write to summary file", as streaming should not snapshot a partially written file.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481913923



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       @maedhroz @dcapwell but then, suddenly, we are perfecty happy to cast that into AsyncStreamingOutputPlus in without any checks and that write method also takes DataOutputStreamPlus ... Same for uncompressed / normal writer.
    https://github.com/apache/cassandra/blob/7542fd81abb445e10fbcc67929d742a9849b7129/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java#L61




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469476960



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2079,30 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate sstable level with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(int newLevel) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateLevel(descriptor, newLevel);
+            reloadSSTableMetadata();
+        };

Review comment:
       nit: remove `;`

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2079,30 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate sstable level with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(int newLevel) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateLevel(descriptor, newLevel);
+            reloadSSTableMetadata();
+        };
+    }
+
+    /**
+     * Mutate sstable repair metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(long newRepairedAt, UUID newPendingRepair, boolean isTransient) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, newRepairedAt, newPendingRepair, isTransient);
+            reloadSSTableMetadata();
+        };

Review comment:
       nit: remove `;`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
blerer commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r485681647



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?

Review comment:
       Good catch :-)
    




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459539460



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);

Review comment:
       added try-with-resources




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473139366



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -430,52 +430,15 @@ public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> c
             System.exit(1);
         }
 
-        long fileLength = new File(descriptor.filenameFor(Component.DATA)).length();
-        logger.info("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength));
-
-        final SSTableReader sstable;
         try
         {
-            sstable = internalOpen(descriptor,
-                                   components,
-                                   metadata,
-                                   System.currentTimeMillis(),
-                                   statsMetadata,
-                                   OpenReason.NORMAL,
-                                   header.toHeader(metadata.get()));
+            BuilderForBatch builder = new BuilderForBatch(descriptor, metadata);
+            return builder.build(components, statsMetadata, OpenReason.NORMAL, header.toHeader(builder.metadata));

Review comment:
       aside: It's kind of nice now that we don't have to even worry about creating a reader instance until the things that it depends on are actually ready/available.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481692234



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,
+                      boolean summaryLoaded,
+                      Set<Component> components,
+                      StatsMetadata statsMetadata) throws IOException
+    {
+        if (!components.contains(Component.PRIMARY_INDEX))
+            return;
+
+        if (logger.isDebugEnabled())
+            logger.debug("Attempting to build summary for {}", descriptor);
+
+
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+        {
+            long indexSize = primaryIndex.length();
+            long histogramCount = statsMetadata.estimatedPartitionSize.count();
+            long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                 ? histogramCount
+                                 : SSTable.estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+            if (recreateBloomFilter)
+                bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+            {
+                long indexPosition;
+
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                    RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                    DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                    if (first == null)
+                        first = decoratedKey;
+                    last = decoratedKey;
+
+                    if (recreateBloomFilter)
+                        bf.add(decoratedKey);
+
+                    // if summary was already read from disk we don't want to re-populate it using primary index
+                    if (!summaryLoaded)
+                    {
+                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
+                    }
+                }
+
+                if (!summaryLoaded)
+                    summary = summaryBuilder.build(metadata.partitioner);
+            }
+        }
+
+        first = SSTable.getMinimalKey(first);
+        last = SSTable.getMinimalKey(last);
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     *
+     * @throws IOException
+     */
+    IFilter loadBloomFilter() throws IOException
+    {
+        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
+        {
+            return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat());
+        }
+    }
+
+    public static class ForWriter extends SSTableReaderBuilder
+    {
+        public ForWriter(Descriptor descriptor,
+                         TableMetadataRef metadataRef,
+                         long maxDataAge,
+                         Set<Component> components,
+                         StatsMetadata statsMetadata,
+                         SSTableReader.OpenReason openReason,
+                         SerializationHeader header)
+        {
+            super(descriptor, metadataRef, maxDataAge, components, statsMetadata, openReason, header);
+        }
+
+        @Override
+        public SSTableReader build()
+        {
+            SSTableReader reader = readerFactory.open(this);
+
+            reader.setup(true);
+            return reader;
+        }
+    }
+
+    public static class ForBatch extends SSTableReaderBuilder
+    {
+        public ForBatch(Descriptor descriptor,
+                        TableMetadataRef metadataRef,
+                        Set<Component> components,
+                        StatsMetadata statsMetadata,
+                        SerializationHeader header)
+        {
+            super(descriptor, metadataRef, System.currentTimeMillis(), components, statsMetadata, SSTableReader.OpenReason.NORMAL, header);
+        }
+
+        @Override
+        public SSTableReader build()
+        {
+            String dataFilePath = descriptor.filenameFor(Component.DATA);
+            long fileLength = new File(dataFilePath).length();
+            logger.info("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength));
+
+            initSummary(dataFilePath, components, statsMetadata);
+
+            boolean compression = components.contains(Component.COMPRESSION_INFO);
+            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
+                    .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+                    .withChunkCache(ChunkCache.instance);
+                    FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)
+                                                                                                                .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
+                                                                                                                .withChunkCache(ChunkCache.instance))
+            {
+                long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+                DiskOptimizationStrategy optimizationStrategy = DatabaseDescriptor.getDiskOptimizationStrategy();
+                int dataBufferSize = optimizationStrategy.bufferSize(statsMetadata.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
+                int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / summary.size());
+                ifile = ibuilder.bufferSize(indexBufferSize).complete();
+                dfile = dbuilder.bufferSize(dataBufferSize).complete();
+                bf = FilterFactory.AlwaysPresent;
+
+                SSTableReader sstable = readerFactory.open(this);
+
+                sstable.first = first;

Review comment:
       @dcapwell Yeah, we had some back and forth on this in one of the resolved comments above and ended in pretty much the same place.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458443215



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -127,6 +135,8 @@ public String getName()
     @Override
     public long getSize()
     {
+        // File size may not be accurate due to concurrent stats update or index summary redistribution. Since it's only
+        // used for metrics, an estimation should be fine.
         return header.size();

Review comment:
       If this doesn't happen much, what about just generating the header on the fly? We'd at least avoid the slight confusion of having both an instance field and the `current` header for entire-SSTable streaming...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459347616



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -127,6 +135,8 @@ public String getName()
     @Override
     public long getSize()
     {
+        // File size may not be accurate due to concurrent stats update or index summary redistribution. Since it's only
+        // used for metrics, an estimation should be fine.
         return header.size();

Review comment:
       I didn't compute size in flight because the size computation is slightly slow for non-entire-streaming with compression..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473656928



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -701,16 +633,24 @@ protected SSTableReader(final Descriptor desc,
                             long maxDataAge,
                             StatsMetadata sstableMetadata,
                             OpenReason openReason,
-                            SerializationHeader header)
+                            SerializationHeader header,
+                            IndexSummary summary,
+                            FileHandle dfile,
+                            FileHandle ifile,
+                            IFilter bf)
     {
         super(desc, components, metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
         this.sstableMetadata = sstableMetadata;
         this.header = header;
+        this.indexSummary = summary;
         this.maxDataAge = maxDataAge;
         this.openReason = openReason;
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), desc.version, header);
         tidy = new InstanceTidier(descriptor, metadata.id);
         selfRef = new Ref<>(this, tidy);
+        this.dfile = dfile;
+        this.ifile = ifile;
+        this.bf = bf;

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
##########
@@ -53,9 +55,11 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(BigTableReader.class);
 
-    BigTableReader(Descriptor desc, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header)
+    BigTableReader(Descriptor desc, Set<Component> components, TableMetadataRef metadata, Long maxDataAge,
+                   StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header, IndexSummary summary,
+                   FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483977041



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?
-                     new CassandraStreamWriter(sstable, header.sections, session) :
-                     new CassandraCompressedStreamWriter(sstable, header.sections,
-                                                         header.compressionInfo, session);
+                                           new CassandraStreamWriter(sstable, header.sections, session) :
+                                           new CassandraCompressedStreamWriter(sstable, header.sections, header.compressionInfo, session);

Review comment:
       good idea, updated to pass `CassandraStreamHeader` into both writes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r464697768



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -63,6 +120,12 @@ public long totalSize()
         return new ArrayList<>(components.keySet());
     }
 
+    @VisibleForTesting
+    public Collection<File> hardlinks()

Review comment:
       @jasonstack Is this unused now?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473132144



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1125,18 +839,47 @@ private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason,
                                                  metadata,
                                                  ifile != null ? ifile.sharedCopy() : null,
                                                  dfile.sharedCopy(),
-                                                 newSummary,
                                                  bf.sharedCopy(),
                                                  maxDataAge,
                                                  sstableMetadata,
                                                  reason,
-                                                 header);
+                                                 header,
+                                                 newSummary);
+
         replacement.first = newFirst;
         replacement.last = last;
         replacement.isSuspect.set(isSuspect.get());
         return replacement;
     }
 
+    /**
+     * Clone this reader with the new values and set the clone as replacement.
+     *
+     * @param newBloomFilter for the replacement
+     *
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    @VisibleForTesting
+    public SSTableReader cloneAndReplace(IFilter newBloomFilter)
+    {
+        SSTableReader replacement = internalOpen(descriptor,
+                                                 components,
+                                                 metadata,
+                                                 ifile != null ? ifile.sharedCopy() : null,

Review comment:
       @jasonstack Should `ifile` ever be `null` here? At least for this version of `cloneAndReplace()`, it looks like the only two things that use it (tests) pass without the null check...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470073171



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I've had a chance to talk it over w/ @belliottsmith and think about it, and tl;dr the changes here should be safe. The big thing I was missing is that the `SSTableReader` itself should already be published safely between threads, and if it is, the `indexSummary` reference should be fine (i.e. not null, which is the only danger, because it doesn't actually change after creation given we clone).
   
   However, it might be helpful to briefly document this in `cloneWithNewSummarySamplingLevel()` so future readers might avoid some confusion. Better yet, we could finally just make `indexSummary` final in `SSTableReader`, avoiding all this confusion in the first place. I'm guessing the reason they aren't is that we added some layers of initialization over time, but deferring reader creation until we're ready seems like it could be achieved with a builder (or even without one).
   
   CC @jasonstack 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458432925



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);

Review comment:
       nit: Rather than a timed wait, what about just counting down on a latch when the provided `Callable` actually starts. (We'd probably have to make it a field in the test class, but it can be reset in a `@Before` block, etc.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473120914



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -626,42 +575,25 @@ public void run()
      * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
      */
     public static SSTableReader internalOpen(Descriptor desc,
-                                      Set<Component> components,
-                                      TableMetadataRef metadata,
-                                      FileHandle ifile,
-                                      FileHandle dfile,
-                                      IndexSummary isummary,
-                                      IFilter bf,
-                                      long maxDataAge,
-                                      StatsMetadata sstableMetadata,
-                                      OpenReason openReason,
-                                      SerializationHeader header)
-    {
-        assert desc != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
-
-        SSTableReader reader = internalOpen(desc, components, metadata, maxDataAge, sstableMetadata, openReason, header);
-
-        reader.bf = bf;
-        reader.ifile = ifile;
-        reader.dfile = dfile;
-        reader.indexSummary = isummary;
-        reader.setup(true);
-
-        return reader;
-    }
-
-
-    private static SSTableReader internalOpen(final Descriptor descriptor,
-                                              Set<Component> components,
-                                              TableMetadataRef metadata,
-                                              Long maxDataAge,
-                                              StatsMetadata sstableMetadata,
-                                              OpenReason openReason,
-                                              SerializationHeader header)
-    {
-        Factory readerFactory = descriptor.getFormat().getReaderFactory();
-
-        return readerFactory.open(descriptor, components, metadata, maxDataAge, sstableMetadata, openReason, header);
+                                             Set<Component> components,
+                                             TableMetadataRef metadata,
+                                             FileHandle ifile,
+                                             FileHandle dfile,
+                                             IFilter bf,
+                                             long maxDataAge,
+                                             StatsMetadata sstableMetadata,
+                                             OpenReason openReason,
+                                             SerializationHeader header,
+                                             IndexSummary summary)
+    {
+        assert desc != null && ifile != null && dfile != null && summary != null && bf != null && sstableMetadata != null;
+
+        BuilderForWriter builder = new BuilderForWriter(desc, metadata, maxDataAge);
+        builder.bf = bf;
+        builder.ifile = ifile;
+        builder.dfile = dfile;
+        builder.summary = summary;

Review comment:
       nit: We could make a minor clean up here and have "chainable" methods for setting the bloom filter if you think that reads better.
   
   ex.
   ```
   BuilderForWriter builder =
       new BuilderForWriter(desc, metadata, maxDataAge).bloomFilter(bf)
                                                       .indexFile(ifile)
                                                       .dataFile(dfile)
                                                       .indexSummary(summary);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-662048956


   @jasonstack FWIW, both of the dtests that failed [here](https://app.circleci.com/pipelines/github/jasonstack/cassandra/244/workflows/9e2af3a1-7b63-423d-8cde-d2cd178c81d6/jobs/2329) passed locally for me.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469471851



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller
+    public FileChannel channel(Descriptor descriptor, Component component, long size) throws IOException
+    {
+        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).getPath() : descriptor.filenameFor(component);
+        FileChannel channel = new RandomAccessFile(toTransfer, "r").getChannel();
+
+        assert size == channel.size() : String.format("Entire sstable streaming expects %s file size to be %s but got %s.",
+                                                      component, size, channel.size());
+        return channel;
+    }
+
+    @Override
+    public void close()
+    {
+        hardLinks.values().forEach(File::delete);

Review comment:
       @jasonstack Should we catch and do something with exceptions that might be thrown from `delete()`? I suppose we could use `FileUtils.deleteWithConfirm()`, then `clear()`, then log any exceptions at warn?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458430312



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);

Review comment:
       @jasonstack Do we need to mutate the repair metadata if `redistributeSummaries()` saves the index summary, which is what we seem to be testing in this case?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r485902290



##########
File path: test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
##########
@@ -149,23 +151,23 @@ public void setupBenchmark() throws IOException
                                                                                                null), entireSSTableStreamHeader, session);
 
             List<Range<Token>> requestedRanges = Arrays.asList(new Range<>(sstable.first.minValue().getToken(), sstable.last.getToken()));
-            partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session);
+            CassandraStreamHeader partialSSTableStreamHeader =
+            CassandraStreamHeader.builder()

Review comment:
       nit: horizontal tab




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473143739



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2430,7 +2187,10 @@ public abstract SSTableReader open(final Descriptor descriptor,
                                            Long maxDataAge,
                                            StatsMetadata sstableMetadata,
                                            OpenReason openReason,
-                                           SerializationHeader header);
+                                           SerializationHeader header,
+                                           IndexSummary summary,
+                                           FileHandle dfile,
+                                           FileHandle ifile, IFilter bf);

Review comment:
       nit: newline for `bf`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
blerer commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r482176596



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,

Review comment:
       The javadoc and the method name are both misleading, as we are refactoring things it might be good to address that. 

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.

Review comment:
       Nit: The original implementation was returning a boolean. The new one does not so the javadoc need to be fixed.
   The code is also loading the `first` and `last` keys with the `indexSummary`. It might makes sense to change the method name and the documentation to make that more obvious. 
   

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       All the `CassandraStreamWriter` implementation will cast `DataOutputStreamPlus` into an `AsyncStreamingOutputPlus`.
   In practice all `OutgoingStreamMessage` should be send through a `FileTask` which will alwas use an `AsyncStreamingOutputPlus`.
   What we should have is an assert that the `DataOutputStreamPlus` is an `AsyncStreamingOutputPlus` as if it is not the case it is a programming error.
     

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?

Review comment:
       The code around `CassandraStreamHeader.compressionInfo` is really misleading. On the sender side `compressinInfo` will be `null` until the header is serialized (the serialization calling `CassandraStreamHeader.calculateCompressionInfo`). Until that point the `CassandraStreamHeader.isCompressed` will return `false` even if the stream should be compressed. `equals` and `hashcode` are also affected by this effect.
   For `CassandraStreamHeader.isCompressed` should check if the `compressionMetadata` is different from `null` and `compressionInfo`can be removed from `equals` and `hashcode` as they already use `compressionMetadata` and `sections` in their computation.
   Regarding the `compressionInfo` field I wonder if it will not be better to simply make it as private and provide an accessor method that instantiate it lazily. It will be slightly less efficient but probably much safer. What do you think?  

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,
+                      boolean summaryLoaded,
+                      Set<Component> components,
+                      StatsMetadata statsMetadata) throws IOException
+    {
+        if (!components.contains(Component.PRIMARY_INDEX))
+            return;
+
+        if (logger.isDebugEnabled())
+            logger.debug("Attempting to build summary for {}", descriptor);
+
+
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+        {
+            long indexSize = primaryIndex.length();
+            long histogramCount = statsMetadata.estimatedPartitionSize.count();
+            long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                 ? histogramCount
+                                 : SSTable.estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+            if (recreateBloomFilter)
+                bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+            {
+                long indexPosition;
+
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                    RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                    DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                    if (first == null)
+                        first = decoratedKey;
+                    last = decoratedKey;

Review comment:
       I know that this code was the same before but does it make sense to recompute the `first` and `last` key if `summaryLoaded` is `true` as it means that they have been read from the `summary` file just before.  

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?
-                     new CassandraStreamWriter(sstable, header.sections, session) :
-                     new CassandraCompressedStreamWriter(sstable, header.sections,
-                                                         header.compressionInfo, session);
+                                           new CassandraStreamWriter(sstable, header.sections, session) :
+                                           new CassandraCompressedStreamWriter(sstable, header.sections, header.compressionInfo, session);

Review comment:
       `CassandraStreamWriter` and `CassandraCompressedStreamWriter` reompute the size that was computed in `CassandraStreamHeader`. Rather than recomputing the size we should pass it from the header.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483977041



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?
-                     new CassandraStreamWriter(sstable, header.sections, session) :
-                     new CassandraCompressedStreamWriter(sstable, header.sections,
-                                                         header.compressionInfo, session);
+                                           new CassandraStreamWriter(sstable, header.sections, session) :
+                                           new CassandraCompressedStreamWriter(sstable, header.sections, header.compressionInfo, session);

Review comment:
       good idea, updated to pass `CassandraStreamHeader` into both writers




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r468524097



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> ComponentManifest.create(sstable, true));

Review comment:
       +1 Added `ComponentContext`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469940059



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I would have to spend a while familiarising myself with this patch (and all present usages) to be sure, but this is probably fine - the plain write will piggy-back on whatever `volatile` write is used to publish the SSTable to other threads. I think in any cases we care about it _should_ piggy-back on `Tidy.setup` before this.  The only possible issue is if we modify the summary while the sstable is already published, and even in this case we're fine because `IndexSummary` is an immutable class (by which I mean its fields are all final), so the worst that can happen is that it is not immediately visible (technically the JMM doesn't guarantee it is ever visible to other threads, but in practice it cannot fail to eventually make it visible)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473143383



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1125,18 +839,47 @@ private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason,
                                                  metadata,
                                                  ifile != null ? ifile.sharedCopy() : null,
                                                  dfile.sharedCopy(),
-                                                 newSummary,
                                                  bf.sharedCopy(),
                                                  maxDataAge,
                                                  sstableMetadata,
                                                  reason,
-                                                 header);
+                                                 header,
+                                                 newSummary);

Review comment:
       nit: I suppose we could minimize the diff here by leaving the argument order (for `newSummary`) the same.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481767213



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
##########
@@ -20,23 +20,148 @@
 import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedHashMap;
+import java.util.List;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.streaming.CassandraStreamHeader.CassandraStreamHeaderSerializer;
 import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.CompressionParams;
+import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.SerializationUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
 public class CassandraStreamHeaderTest
 {
+    public static final String KEYSPACE = "CassandraStreamHeaderTest";
+    public static final String CF_COMPRESSED = "compressed";
+
+    private static SSTableReader sstable;
+    private static ColumnFamilyStore store;
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_COMPRESSED).compression(CompressionParams.DEFAULT));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore(CF_COMPRESSED);
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+    }
+
+    @Test
+    public void transferedSizeWithCompressionTest()

Review comment:
       Verified that this fails without the fix in `calculateSize()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471232281



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       `ant eclipse-warnings` didn't complain..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469940059



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I would have to spend a while familiarising myself with this patch (and all present usages) to be sure, and all I've done to answer is skim the current patch, but this is probably fine. Any plain write that occurs before the SSTable is published will piggy-back on the `volatile` write used to publish the SSTable to other threads. I think in any cases we care about it _should_ piggy-back on `Tidy.setup` before this.  
   
   The only likely issue, then, is if we were to modify the summary while the sstable is already published.  Even in this case we're fine because `IndexSummary` is an immutable class (by which I mean its fields are all final), so the worst that can happen is that it is not immediately visible (technically the JMM doesn't guarantee it is ever visible to other threads, but in practice it cannot fail to eventually make it visible)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r460680259



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -30,16 +35,31 @@
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.streaming.StreamManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static java.lang.Math.min;
 
 public final class ComponentManifest implements Iterable<Component>

Review comment:
       I have moved the streaming code out of `ComponentManifest`..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458435968



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);

Review comment:
       nit: Do we need to close the channel/output?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459539660



##########
File path: src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
##########
@@ -164,8 +164,7 @@ public synchronized void add(SSTableReader reader)
             try
             {
                 logger.debug("Could not add sstable {} in level {} - dropping to 0", reader, reader.getSSTableLevel());
-                reader.descriptor.getMetadataSerializer().mutateLevel(reader.descriptor, 0);
-                reader.reloadSSTableMetadata();
+                reader.mutateAndReloadStats(stats -> stats.mutateLevel(0));

Review comment:
       reverted to preload stats component




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481913923



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       @maedhroz @dcapwell but then, suddenly, we are perfectly happy to cast that into `AsyncStreamingOutputPlus` without any checks and that write method also takes DataOutputStreamPlus ... Same for uncompressed / normal writer.
    https://github.com/apache/cassandra/blob/7542fd81abb445e10fbcc67929d742a9849b7129/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java#L61




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481853300



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473975427



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
##########
@@ -103,9 +104,11 @@ public SSTableWriter open(Descriptor descriptor,
     static class ReaderFactory extends SSTableReader.Factory
     {
         @Override
-        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata,
+                                  Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason,
+                                  SerializationHeader header, IndexSummary summary, FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       I had actually almost suggested that earlier. WFM




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r485995937



##########
File path: test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
##########
@@ -149,23 +151,23 @@ public void setupBenchmark() throws IOException
                                                                                                null), entireSSTableStreamHeader, session);
 
             List<Range<Token>> requestedRanges = Arrays.asList(new Range<>(sstable.first.minValue().getToken(), sstable.last.getToken()));
-            partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session);
+            CassandraStreamHeader partialSSTableStreamHeader =
+            CassandraStreamHeader.builder()

Review comment:
       +1 updated format




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483975790



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java
##########
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SSTableReaderBuilder
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderBuilder.class);
+
+    protected final SSTableReader.Factory readerFactory;
+    protected final Descriptor descriptor;
+    protected final TableMetadataRef metadataRef;
+    protected final TableMetadata metadata;
+    protected final long maxDataAge;
+    protected final Set<Component> components;
+    protected final StatsMetadata statsMetadata;
+    protected final SSTableReader.OpenReason openReason;
+    protected final SerializationHeader header;
+
+    protected IndexSummary summary;
+    protected DecoratedKey first;
+    protected DecoratedKey last;
+    protected IFilter bf;
+    protected FileHandle ifile;
+    protected FileHandle dfile;
+
+    public SSTableReaderBuilder(Descriptor descriptor,
+                                TableMetadataRef metadataRef,
+                                long maxDataAge,
+                                Set<Component> components,
+                                StatsMetadata statsMetadata,
+                                SSTableReader.OpenReason openReason,
+                                SerializationHeader header)
+    {
+        this.descriptor = descriptor;
+        this.metadataRef = metadataRef;
+        this.metadata = metadataRef.get();
+        this.maxDataAge = maxDataAge;
+        this.components = components;
+        this.statsMetadata = statsMetadata;
+        this.openReason = openReason;
+        this.header = header;
+        this.readerFactory = descriptor.getFormat().getReaderFactory();
+    }
+
+    public abstract SSTableReader build();
+
+    public SSTableReaderBuilder dfile(FileHandle dfile)
+    {
+        this.dfile = dfile;
+        return this;
+    }
+
+    public SSTableReaderBuilder ifile(FileHandle ifile)
+    {
+        this.ifile = ifile;
+        return this;
+    }
+
+    public SSTableReaderBuilder bf(IFilter bf)
+    {
+        this.bf = bf;
+        return this;
+    }
+
+    public SSTableReaderBuilder summary(IndexSummary summary)
+    {
+        this.summary = summary;
+        return this;
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     */
+    void loadSummary()
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+            return;
+        }
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+        }
+        catch (IOException e)
+        {
+            if (summary != null)
+                summary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     */
+    void buildSummary(boolean recreateBloomFilter,
+                      boolean summaryLoaded,
+                      Set<Component> components,
+                      StatsMetadata statsMetadata) throws IOException
+    {
+        if (!components.contains(Component.PRIMARY_INDEX))
+            return;
+
+        if (logger.isDebugEnabled())
+            logger.debug("Attempting to build summary for {}", descriptor);
+
+
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+        {
+            long indexSize = primaryIndex.length();
+            long histogramCount = statsMetadata.estimatedPartitionSize.count();
+            long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                 ? histogramCount
+                                 : SSTable.estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+            if (recreateBloomFilter)
+                bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+            {
+                long indexPosition;
+
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                    RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                    DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                    if (first == null)
+                        first = decoratedKey;
+                    last = decoratedKey;

Review comment:
       updated, now guarded with `if (!summaryLoaded)`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458442864



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -127,6 +135,8 @@ public String getName()
     @Override
     public long getSize()

Review comment:
       nit: Perhaps we should just rename it `getEstimatedSize()`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471602737



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       Odd. Perhaps it was just an IDEA issue.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458435682



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,14 +122,23 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
         AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);

Review comment:
       Minor nit on the `Queue` below...could be `Queue<?>` to avoid a compiler warning.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-665341160


   @jasonstack btw, there's a random unnecessary semicolon in `SSTableReader#saveSummary()`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481854091



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(ComponentContext.class);
+
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    public FileChannel channel(Descriptor descriptor, Component component, long size) throws IOException
+    {
+        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).getPath() : descriptor.filenameFor(component);
+        @SuppressWarnings("resource") // file channel will be closed by Caller
+        FileChannel channel = new RandomAccessFile(toTransfer, "r").getChannel();
+
+        assert size == channel.size() : String.format("Entire sstable streaming expects %s file size to be %s but got %s.",
+                                                      component, size, channel.size());
+        return channel;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accumulate = null;
+        for (File file : hardLinks.values())
+            accumulate = FileUtils.deleteWithConfirm(file, accumulate);
+
+        hardLinks.clear();
+
+        if (accumulate != null)
+            logger.warn("Failed to remove hard link files: {}", accumulate.getMessage());

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458427678



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))

Review comment:
       Should `UNKNOWN` be `INDEX_SUMMARY`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473657580



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
##########
@@ -103,9 +104,11 @@ public SSTableWriter open(Descriptor descriptor,
     static class ReaderFactory extends SSTableReader.Factory
     {
         @Override
-        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata,
+                                  Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason,
+                                  SerializationHeader header, IndexSummary summary, FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       I extract `Builder` into a separate class `SSTableReaderBuilder` and pass it as the only argument to `open`.. WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470038096



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       So, for the result of `cloneWithNewSummaryAtLevel` to ever be visible to anybody, it must be published safely - in this case ordinarily through the DataTracker.View. So anything we do to provide visibility earlier than this is just providing is some nicer feels, which is fine, but as it happens we have several such volatile writes: the synchronised block when saving the new summary provides visibility, as does the write to `tidy.setup` (which is literally its purpose, iirc, though for the scope of `tidy`).
   
   I am generally in favour of making more explicit links between a visibility requirement and where it is enforced, but there's definitely no need to introduce extra synchronised blocks - a simple write to a volatile property is sufficient, and in this case a comment indicating we're piggy-backing on any of these mentioned volatile writes would be sufficient - as would none of them, since we must publish the sstable safely anyway.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458430856



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+
+            writer.write(out);
+            return mutation;
+        });
+
+        // wait for concurrent component mutation to finish

Review comment:
       nit: For the mutation and the write to finish?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471086069



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2079,30 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate sstable level with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(int newLevel) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateLevel(descriptor, newLevel);
+            reloadSSTableMetadata();
+        };
+    }
+
+    /**
+     * Mutate sstable repair metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(long newRepairedAt, UUID newPendingRepair, boolean isTransient) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, newRepairedAt, newPendingRepair, isTransient);
+            reloadSSTableMetadata();
+        };

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2079,30 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate sstable level with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     */
+    public void mutateAndReloadStats(int newLevel) throws IOException
+    {
+        synchronized (tidy.global)
+        {
+            descriptor.getMetadataSerializer().mutateLevel(descriptor, newLevel);
+            reloadSSTableMetadata();
+        };

Review comment:
       +1

##########
File path: test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
##########
@@ -40,12 +40,7 @@
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader;
-import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamWriter;
-import org.apache.cassandra.db.streaming.CassandraOutgoingFile;
-import org.apache.cassandra.db.streaming.CassandraStreamHeader;
-import org.apache.cassandra.db.streaming.CassandraStreamReader;
-import org.apache.cassandra.db.streaming.CassandraStreamWriter;
+import org.apache.cassandra.db.streaming.*;

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller
+    public FileChannel channel(Descriptor descriptor, Component component, long size) throws IOException
+    {
+        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).getPath() : descriptor.filenameFor(component);
+        FileChannel channel = new RandomAccessFile(toTransfer, "r").getChannel();
+
+        assert size == channel.size() : String.format("Entire sstable streaming expects %s file size to be %s but got %s.",
+                                                      component, size, channel.size());
+        return channel;
+    }
+
+    @Override
+    public void close()
+    {
+        hardLinks.values().forEach(File::delete);

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       using `resource` seems cleaner than `IOResourceOpenedButNotSafelyClosed`..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r480474987



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary

Review comment:
       `s/becaue/because/`

##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(ComponentContext.class);
+
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    public FileChannel channel(Descriptor descriptor, Component component, long size) throws IOException
+    {
+        String toTransfer = hardLinks.containsKey(component) ? hardLinks.get(component).getPath() : descriptor.filenameFor(component);
+        @SuppressWarnings("resource") // file channel will be closed by Caller
+        FileChannel channel = new RandomAccessFile(toTransfer, "r").getChannel();
+
+        assert size == channel.size() : String.format("Entire sstable streaming expects %s file size to be %s but got %s.",
+                                                      component, size, channel.size());
+        return channel;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accumulate = null;
+        for (File file : hardLinks.values())
+            accumulate = FileUtils.deleteWithConfirm(file, accumulate);
+
+        hardLinks.clear();
+
+        if (accumulate != null)
+            logger.warn("Failed to remove hard link files: {}", accumulate.getMessage());

Review comment:
       should be `logger.warn("Failed to remove hard link files", accumulate);`

##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(ComponentContext.class);
+
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);

Review comment:
       any reason to use `LinkedHashMap` rather than `HashMap`?  this allocates a bit more memory and don't see a need for order (we delete files in the same order we created them, but don't think we require that property)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-684340690


   > a slow write to SUMMARY (aka IndexSummaryRedistribution) will block updates to STATS (aka compaction) even though they do not logically conflict; 
   
   index summary redistribution and stats-mutation shouldn't happen together because both are running as part of compactions.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459598427



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java
##########
@@ -119,6 +119,11 @@ private long calculateSize()
             for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
                 transferSize += chunk.length + 4; // 4 bytes for CRC
         }
+        // Delay the creation of compressionInfo at sender to reduce GC pressure, see CASSANDRA-10680.
+        else if (compressionMetadata != null)

Review comment:
       this was accidentally removed in storage engine refactor..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-676522785


   unrelated to patch nit: `GlobalTidy.NULL` should be `final`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458444497



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1199,7 +1219,15 @@ else if (samplingLevel < indexSummary.getSamplingLevel())
             }
 
             // Always save the resampled index
-            saveSummary(newSummary);
+            tidy.global.writeLock.lock();
+            try
+            {
+                saveSummary(newSummary);
+            }
+            finally
+            {
+                tidy.global.writeLock.unlock();
+            }

Review comment:
       What do you think about having a `runWithWriteLock(Runnable)` or something similar on `GlobalTidy`? I'm guessing we're not concerned about the overhead of crating a lambda, etc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r483979368



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?

Review comment:
       > CassandraStreamHeader.isCompressed will return false even if the stream should be compressed.
   
   `isCompressed()` is used on received side where `compressionMetadata` is always `null` regardless compressed. But good to make it consistent.
   
   > Regarding the compressionInfo field I wonder if it will not be better to simply make it as private and provide an accessor method that instantiate it lazily.
   
   Good idea, added accessor `getOrInitCompressionInfo`
   
   > compressionInfo can be removed from equals and hashcode as they already use compressionMetadata and sections in their computation.
   
   `equals`/`hashCode` didn't use `compressionMetadata`. Since `equals/hashCode` is not used in production code, I have changed them to use `getOrInitCompressionInfo()`, wdyt?
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459342969



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -127,6 +135,8 @@ public String getName()
     @Override
     public long getSize()

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458430009



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1199,7 +1219,15 @@ else if (samplingLevel < indexSummary.getSamplingLevel())
             }
 
             // Always save the resampled index
-            saveSummary(newSummary);
+            tidy.global.writeLock.lock();
+            try
+            {
+                saveSummary(newSummary);

Review comment:
       @jasonstack It looks like this is the only summary saving path that could affect streaming, but wanted to make sure...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r480863900



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       I believe this `out instaceof` is useless, if you check how that `out` is instantiated, it can never be something else.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471142146



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I made some refactor based on @maedhroz 's builder prototype: https://github.com/jasonstack/cassandra/pull/16. Now we have `BuilderForWriter`/`BuilderForRead`/`BuilderForBatch` and `IndexSummary` is `final`. We can make `dfile/ifile/bf` final as well, but didn't do it in this commit..
   
   what do you guys think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473667151



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1125,18 +839,47 @@ private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason,
                                                  metadata,
                                                  ifile != null ? ifile.sharedCopy() : null,
                                                  dfile.sharedCopy(),
-                                                 newSummary,
                                                  bf.sharedCopy(),
                                                  maxDataAge,
                                                  sstableMetadata,
                                                  reason,
-                                                 header);
+                                                 header,
+                                                 newSummary);
+
         replacement.first = newFirst;
         replacement.last = last;
         replacement.isSuspect.set(isSuspect.get());
         return replacement;
     }
 
+    /**
+     * Clone this reader with the new values and set the clone as replacement.
+     *
+     * @param newBloomFilter for the replacement
+     *
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    @VisibleForTesting
+    public SSTableReader cloneAndReplace(IFilter newBloomFilter)
+    {
+        SSTableReader replacement = internalOpen(descriptor,
+                                                 components,
+                                                 metadata,
+                                                 ifile != null ? ifile.sharedCopy() : null,

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -626,42 +575,25 @@ public void run()
      * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
      */
     public static SSTableReader internalOpen(Descriptor desc,
-                                      Set<Component> components,
-                                      TableMetadataRef metadata,
-                                      FileHandle ifile,
-                                      FileHandle dfile,
-                                      IndexSummary isummary,
-                                      IFilter bf,
-                                      long maxDataAge,
-                                      StatsMetadata sstableMetadata,
-                                      OpenReason openReason,
-                                      SerializationHeader header)
-    {
-        assert desc != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
-
-        SSTableReader reader = internalOpen(desc, components, metadata, maxDataAge, sstableMetadata, openReason, header);
-
-        reader.bf = bf;
-        reader.ifile = ifile;
-        reader.dfile = dfile;
-        reader.indexSummary = isummary;
-        reader.setup(true);
-
-        return reader;
-    }
-
-
-    private static SSTableReader internalOpen(final Descriptor descriptor,
-                                              Set<Component> components,
-                                              TableMetadataRef metadata,
-                                              Long maxDataAge,
-                                              StatsMetadata sstableMetadata,
-                                              OpenReason openReason,
-                                              SerializationHeader header)
-    {
-        Factory readerFactory = descriptor.getFormat().getReaderFactory();
-
-        return readerFactory.open(descriptor, components, metadata, maxDataAge, sstableMetadata, openReason, header);
+                                             Set<Component> components,
+                                             TableMetadataRef metadata,
+                                             FileHandle ifile,
+                                             FileHandle dfile,
+                                             IFilter bf,
+                                             long maxDataAge,
+                                             StatsMetadata sstableMetadata,
+                                             OpenReason openReason,
+                                             SerializationHeader header,
+                                             IndexSummary summary)
+    {
+        assert desc != null && ifile != null && dfile != null && summary != null && bf != null && sstableMetadata != null;
+
+        BuilderForWriter builder = new BuilderForWriter(desc, metadata, maxDataAge);
+        builder.bf = bf;
+        builder.ifile = ifile;
+        builder.dfile = dfile;
+        builder.summary = summary;

Review comment:
       good idea.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459342830



##########
File path: src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
##########
@@ -1200,8 +1200,7 @@ public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt,
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
-                sstable.reloadSSTableMetadata();
+                sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(repairedAt, pendingRepair, isTransient));

Review comment:
       I can put the trace back.. or revert the change if stats component is preloaded.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459526554



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+
+            writer.write(out);
+            return mutation;
+        });
+
+        // wait for concurrent component mutation to finish

Review comment:
       `writer.write` is blocking, it's already finished.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r468524578



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -63,6 +120,12 @@ public long totalSize()
         return new ArrayList<>(components.keySet());
     }
 
+    @VisibleForTesting
+    public Collection<File> hardlinks()

Review comment:
       removed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r471086077



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       moved `@SuppressWarnings ` to `RandomAccessFile `, but using `resource` seems cleaner than `IOResourceOpenedButNotSafelyClosed`..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459537258



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+
+            writer.write(out);
+            return mutation;
+        });
+
+        // wait for concurrent component mutation to finish
+        future.get(15, TimeUnit.SECONDS);
 
         session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104));
 
-        CassandraStreamHeader header =
-            CassandraStreamHeader.builder()
-                                 .withSSTableFormat(sstable.descriptor.formatType)
-                                 .withSSTableVersion(sstable.descriptor.version)
-                                 .withSSTableLevel(0)
-                                 .withEstimatedKeys(sstable.estimatedKeys())
-                                 .withSections(Collections.emptyList())
-                                 .withSerializationHeader(sstable.header.toComponent())
-                                 .withComponentManifest(CassandraOutgoingFile.getComponentManifest(sstable))
-                                 .isEntireSSTable(true)
-                                 .withFirstKey(sstable.first)
-                                 .withTableId(sstable.metadata().id)
-                                 .build();
-
-        CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null), header, session);
+        StreamMessageHeader messageHeader = new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null);
+        CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(messageHeader, header, session);
 
         SSTableMultiWriter sstableWriter = reader.read(new DataInputBuffer(serializedFile.nioBuffer(), false));

Review comment:
       good idea




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459536388



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481768291



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
##########
@@ -20,23 +20,148 @@
 import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedHashMap;
+import java.util.List;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.streaming.CassandraStreamHeader.CassandraStreamHeaderSerializer;
 import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.CompressionParams;
+import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.SerializationUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
 public class CassandraStreamHeaderTest
 {
+    public static final String KEYSPACE = "CassandraStreamHeaderTest";
+    public static final String CF_COMPRESSED = "compressed";
+
+    private static SSTableReader sstable;
+    private static ColumnFamilyStore store;
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_COMPRESSED).compression(CompressionParams.DEFAULT));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore(CF_COMPRESSED);
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+    }
+
+    @Test
+    public void transferedSizeWithCompressionTest()

Review comment:
       CC @blerer @smiklosovic 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r472924302



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       updated the PR to make `dfile/ifile/bf` final.. `first/last` are used in `SSTable` and modified by writer.. I will leave them as they are..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
blerer commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r484935231



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?

Review comment:
       I missed the receiving side logic with `compressionMetadata` vs `compressionInfo`. The changes you did will work, I just wonder if we cannot make the logic even less confusing. I tried to refactor things a bit and came up with:
   https://github.com/apache/cassandra/commit/5c01763f7f8ae6b58a2d795101465d8195bf56d4
   Tell me if it makes sense to you. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458438552



##########
File path: src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
##########
@@ -164,8 +164,7 @@ public synchronized void add(SSTableReader reader)
             try
             {
                 logger.debug("Could not add sstable {} in level {} - dropping to 0", reader, reader.getSSTableLevel());
-                reader.descriptor.getMetadataSerializer().mutateLevel(reader.descriptor, 0);
-                reader.reloadSSTableMetadata();
+                reader.mutateAndReloadStats(stats -> stats.mutateLevel(0));

Review comment:
       Same as above. Does this mean we no longer trace like we would have in `IMetadataSerializer#mutateLevel()`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r485574912



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation because of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest and hard links for mutatable components in case they are modified.
+            try (ComponentContext context = sstable.runWithLock(ignored -> ComponentContext.create(sstable.descriptor)))
+            {
+                CassandraStreamHeader current = makeHeader(sstable, operation, sections, estimatedKeys, true, context.manifest());
+                CassandraStreamHeader.serializer.serialize(current, out, version);
+                out.flush();
+
+                CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, context);
+                writer.write((AsyncStreamingOutputPlus) out);
+            }
         }
         else
         {
+            // legacy streaming is not affected by stats metadata mutation and index sumary redistribution
+            CassandraStreamHeader.serializer.serialize(header, out, version);
+            out.flush();
+
             CassandraStreamWriter writer = (header.compressionInfo == null) ?

Review comment:
       Thanks for the refactoring, it looks great. I made a minor fix to use `Arrays.equals` for `chunks()` in `CompressionInfo#equals` https://github.com/apache/cassandra/pull/642/commits/7522f57e9397d94820a1020c1bba808b8a262e1a




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473146437



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2538,4 +2298,369 @@ public static void shutdownBlocking(long timeout, TimeUnit unit) throws Interrup
         ExecutorUtils.shutdownNowAndWait(timeout, unit, syncExecutor);
         resetTidying();
     }
+    
+    private static abstract class Builder
+    {
+        protected final Descriptor descriptor;
+        protected final TableMetadataRef metadataRef;
+        protected final TableMetadata metadata;
+        protected final Factory readerFactory;
+
+        public IndexSummary summary;
+        public DecoratedKey first;
+        public DecoratedKey last;
+        public IFilter bf;
+        public FileHandle ifile;
+        public FileHandle dfile;
+
+        public Builder(Descriptor descriptor, TableMetadataRef metadataRef)
+        {
+            this.descriptor = descriptor;
+            this.metadataRef = metadataRef;
+            this.metadata = metadataRef.get();
+            this.readerFactory = descriptor.getFormat().getReaderFactory();
+        }
+
+        /**
+         * Load index summary from Summary.db file if it exists.
+         *
+         * if loaded index summary has different index interval from current value stored in schema,
+         * then Summary.db file will be deleted and this returns false to rebuild summary.
+         */
+        void loadSummary()
+        {
+            File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+            if (!summariesFile.exists())
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("SSTable Summary File {} does not exist", summariesFile.getAbsolutePath());
+                return;
+            }
+
+            DataInputStream iStream = null;
+            try
+            {
+                iStream = new DataInputStream(Files.newInputStream(summariesFile.toPath()));
+                summary = IndexSummary.serializer.deserialize(iStream,
+                                                              metadata.partitioner,
+                                                              metadata.params.minIndexInterval,
+                                                              metadata.params.maxIndexInterval);
+                first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+                last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            }
+            catch (IOException e)
+            {
+                if (summary != null)
+                    summary.close();
+                logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+                // corrupted; delete it and fall back to creating a new summary
+                FileUtils.closeQuietly(iStream);
+                // delete it and fall back to creating a new summary
+                FileUtils.deleteWithConfirm(summariesFile);
+            }
+            finally
+            {
+                FileUtils.closeQuietly(iStream);
+            }
+        }
+
+        /**
+         * Build index summary(and optionally bloom filter) by reading through Index.db file.
+         *
+         * @param recreateBloomFilter true if recreate bloom filter
+         * @param summaryLoaded true if index summary is already loaded and not need to build again
+         */
+        void buildSummary(boolean recreateBloomFilter,
+                          boolean summaryLoaded,
+                          Set<Component> components,
+                          StatsMetadata statsMetadata) throws IOException
+        {
+            if (!components.contains(Component.PRIMARY_INDEX))
+                return;
+
+            if (logger.isDebugEnabled())
+                logger.debug("Attempting to build summary for {}", descriptor);
+
+
+            // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+            try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+            {
+                long indexSize = primaryIndex.length();
+                long histogramCount = statsMetadata.estimatedPartitionSize.count();
+                long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed()
+                                     ? histogramCount
+                                     : estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional
+
+                if (recreateBloomFilter)
+                    bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance);
+
+                try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL))
+                {
+                    long indexPosition;
+
+                    while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                    {
+                        ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                        RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version);
+                        DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key);
+                        if (first == null)
+                            first = decoratedKey;
+                        last = decoratedKey;
+
+                        if (recreateBloomFilter)
+                            bf.add(decoratedKey);
+
+                        // if summary was already read from disk we don't want to re-populate it using primary index
+                        if (!summaryLoaded)
+                        {
+                            summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
+                        }
+                    }
+
+                    if (!summaryLoaded)
+                        summary = summaryBuilder.build(metadata.partitioner);
+                }
+            }
+
+            first = getMinimalKey(first);
+            last = getMinimalKey(last);
+        }
+
+        /**
+         * Load bloom filter from Filter.db file.
+         *
+         * @throws IOException
+         */
+        IFilter loadBloomFilter() throws IOException
+        {
+            try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
+            {
+                return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat());
+            }
+        }
+
+        public abstract SSTableReader build(Set<Component> components, StatsMetadata statsMetadata, OpenReason reason, SerializationHeader header);

Review comment:
       Fun Fact: Everything would compile if we removed this abstract method ;)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459347616



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -127,6 +135,8 @@ public String getName()
     @Override
     public long getSize()
     {
+        // File size may not be accurate due to concurrent stats update or index summary redistribution. Since it's only
+        // used for metrics, an estimation should be fine.
         return header.size();

Review comment:
       I didn't compute size on the fly because the size computation is slightly slow for non-entire-streaming with compression..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459524140



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1199,7 +1219,15 @@ else if (samplingLevel < indexSummary.getSamplingLevel())
             }
 
             // Always save the resampled index
-            saveSummary(newSummary);
+            tidy.global.writeLock.lock();
+            try
+            {
+                saveSummary(newSummary);
+            }
+            finally
+            {
+                tidy.global.writeLock.unlock();
+            }

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r461320393



##########
File path: src/java/org/apache/cassandra/streaming/OutgoingStream.java
##########
@@ -47,7 +47,12 @@
     UUID getPendingRepair();
 
     String getName();
-    long getSize();
+
+    /**
+     * @return estimated file size to be streamed. This should only be used for metrics only, because concurrent

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -55,55 +56,79 @@
     private final long estimatedKeys;
     private final List<SSTableReader.PartitionPositionBounds> sections;
     private final String filename;
-    private final CassandraStreamHeader header;
-    private final boolean keepSSTableLevel;
-    private final ComponentManifest manifest;
-
     private final boolean shouldStreamEntireSSTable;
+    private final StreamOperation operation;
+    private final CassandraStreamHeader header;
 
     public CassandraOutgoingFile(StreamOperation operation, Ref<SSTableReader> ref,
                                  List<SSTableReader.PartitionPositionBounds> sections, List<Range<Token>> normalizedRanges,
                                  long estimatedKeys)
     {
         Preconditions.checkNotNull(ref.get());
         Range.assertNormalized(normalizedRanges);
+        this.operation = operation;
         this.ref = ref;
         this.estimatedKeys = estimatedKeys;
         this.sections = sections;
-        this.filename = ref.get().getFilename();
-        this.manifest = getComponentManifest(ref.get());
-        this.shouldStreamEntireSSTable = computeShouldStreamEntireSSTables();
 
         SSTableReader sstable = ref.get();
-        keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD;
-        this.header =
-            CassandraStreamHeader.builder()
-                                 .withSSTableFormat(sstable.descriptor.formatType)
-                                 .withSSTableVersion(sstable.descriptor.version)
-                                 .withSSTableLevel(keepSSTableLevel ? sstable.getSSTableLevel() : 0)
-                                 .withEstimatedKeys(estimatedKeys)
-                                 .withSections(sections)
-                                 .withCompressionMetadata(sstable.compression ? sstable.getCompressionMetadata() : null)
-                                 .withSerializationHeader(sstable.header.toComponent())
-                                 .isEntireSSTable(shouldStreamEntireSSTable)
-                                 .withComponentManifest(manifest)
-                                 .withFirstKey(sstable.first)
-                                 .withTableId(sstable.metadata().id)
-                                 .build();
+
+        this.filename = sstable.getFilename();
+        this.shouldStreamEntireSSTable = computeShouldStreamEntireSSTables();
+        ComponentManifest manifest = getComponentManifest(sstable, false);
+        this.header = makeHeader(sstable, operation, sections, estimatedKeys, shouldStreamEntireSSTable, manifest);
+    }
+
+    private static CassandraStreamHeader makeHeader(SSTableReader sstable,
+                                                    StreamOperation operation,
+                                                    List<SSTableReader.PartitionPositionBounds> sections,
+                                                    long estimatedKeys,
+                                                    boolean shouldStreamEntireSSTable,
+                                                    ComponentManifest manifest)
+    {
+        boolean keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD;
+
+        return CassandraStreamHeader.builder()
+                                    .withSSTableFormat(sstable.descriptor.formatType)
+                                    .withSSTableVersion(sstable.descriptor.version)
+                                    .withSSTableLevel(keepSSTableLevel ? sstable.getSSTableLevel() : 0)
+                                    .withEstimatedKeys(estimatedKeys)
+                                    .withSections(sections)
+                                    .withCompressionMetadata(sstable.compression ? sstable.getCompressionMetadata() : null)
+                                    .withSerializationHeader(sstable.header.toComponent())
+                                    .isEntireSSTable(shouldStreamEntireSSTable)
+                                    .withComponentManifest(manifest)
+                                    .withFirstKey(sstable.first)
+                                    .withTableId(sstable.metadata().id)
+                                    .build();
     }
 
     @VisibleForTesting
-    public static ComponentManifest getComponentManifest(SSTableReader sstable)
+    public static ComponentManifest getComponentManifest(SSTableReader sstable, boolean createHardLink)

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2078,25 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate Stats Metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     *
+     * @param transform function to mutate stats metadata
+     */
+    public void mutateAndReloadStats(UnaryOperator<StatsMetadata> transform) throws IOException
+    {
+        tidy.global.writeLock.lock();
+        try
+        {
+            descriptor.getMetadataSerializer().mutate(descriptor, transform);
+            reloadSSTableMetadata();
+        }
+        finally
+        {
+            tidy.global.writeLock.unlock();
+        }

Review comment:
       +1, missed it when reverting code..

##########
File path: test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java
##########
@@ -127,7 +127,7 @@ public void test() throws Exception
 
         int totalNumberOfFiles = session.transfers.get(store.metadata.id).getTotalNumberOfFiles();
 
-        assertEquals(CassandraOutgoingFile.getComponentManifest(sstable).components().size(), totalNumberOfFiles);
+        assertEquals(CassandraOutgoingFile.getComponentManifest(sstable, false).components().size(), totalNumberOfFiles);

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +179,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> getComponentManifest(sstable, true));

Review comment:
       so the index summary hark link is created on either old summary or new summary, never on the partially written file because of the lock..

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       agreed, we need another test to verify the lock. These tests in `CassandraEntireSSTableStreamWriter` only verify hard links are working as expected (`hard links` are not impacted by replace/deletion).

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       agreed, we need another test to verify the lock. These tests in `CassandraEntireSSTableStreamWriter` only verify hard links are working as expected (`hard links` are not impacted by replace/deletion).

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +179,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> getComponentManifest(sstable, true));

Review comment:
       ah.. forgot to hard link `index summary`..

##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +179,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> getComponentManifest(sstable, true));

Review comment:
       so the index summary hark link is created on either old summary or new summary, never on the partially written file because of the lock..

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -105,69 +124,144 @@ public static void defineSchemaAndPrepareSSTable()
         CompactionManager.instance.performMaximal(store, false);
 
         sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+    }
+
+    @Before
+    public void init()
+    {
+        latch = new CountDownLatch(1);
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
     @Test
     public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, false);
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
 
         EmbeddedChannel channel = new EmbeddedChannel();
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
-        writer.write(out);
+        try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            writer.write(out);
 
-        Queue msgs = channel.outboundMessages();
+            Queue<?> msgs = channel.outboundMessages();
 
-        assertTrue(msgs.peek() instanceof DefaultFileRegion);
+            assertTrue(msgs.peek() instanceof DefaultFileRegion);
+        }
     }
 
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            latch.countDown();
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(Collections.singletonList(sstable), OperationType.INDEX_SUMMARY))
+            {
+                latch.countDown();
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
 
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable, true);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait for concurrent task to start
+                waitForLatch(10, TimeUnit.SECONDS);

Review comment:
       agreed, we need some tests for locks. `CassandraEntireSSTableStreamWriter` only verifies hard links are working as expected, ie. `not affected by file deletion or replacement`

##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -112,7 +111,7 @@ public void testBlockWriterOverWire() throws IOException
     {
         StreamSession session = setupStreamingSessionForTest();
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, ComponentManifest.create(sstable, false));
 
         EmbeddedChannel channel = new EmbeddedChannel();

Review comment:
       +1

##########
File path: src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
##########
@@ -1200,8 +1200,7 @@ public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt,
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
-                sstable.reloadSSTableMetadata();
+                sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(repairedAt, pendingRepair, isTransient));

Review comment:
       added an `description` argument to pass trace info

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")

Review comment:
       +1

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")
+    public void testStreamWithIndexSummaryRedistributionDelaySavingSummary() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(() -> {
+            // wait until new index summary is partially written
+            latch.await(1, TimeUnit.MINUTES);
+            return null;
+        }, this::indexSummaryRedistribution);
+    }
+
+    // used by byteman
+    private static void countDown()
+    {
+        latch.countDown();
+    }

Review comment:
       +1

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),

Review comment:
       +1

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")
+    public void testStreamWithIndexSummaryRedistributionDelaySavingSummary() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(() -> {
+            // wait until new index summary is partially written
+            latch.await(1, TimeUnit.MINUTES);
+            return null;
+        }, this::indexSummaryRedistribution);
+    }
+
+    // used by byteman
+    private static void countDown()
+    {
+        latch.countDown();
+    }
+    private void testStreamWithConcurrentComponentMutation(Callable<?> runBeforeStreaming, Callable<?> mutation) throws Exception
+    {
+        ByteBuf serializedFile = Unpooled.buffer(8192);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        Collection<OutgoingStream> outgoingStreams = store.getStreamManager().createOutgoingStreams(session, rangesAtEndpoint, NO_PENDING_REPAIR, PreviewKind.NONE);
+        CassandraOutgoingFile outgoingFile = (CassandraOutgoingFile) Iterables.getOnlyElement(outgoingStreams);
+
+        Future<?> streaming = executeAsync(() -> {
+            runBeforeStreaming.call();
+
+            try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(createMockNettyChannel(serializedFile)))
+            {
+                outgoingFile.write(session, out, MessagingService.current_version);
+                assertTrue(sstable.descriptor.getTemporaryFiles().isEmpty());
+            }
+            return null;
+        });
+
+        Future<?> concurrentMutations = executeAsync(mutation);
+
+        streaming.get(3, TimeUnit.MINUTES);
+        concurrentMutations.get(3, TimeUnit.MINUTES);
+
+        session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104));
+        StreamMessageHeader messageHeader = new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null);
+
+        try (DataInputBuffer in = new DataInputBuffer(serializedFile.nioBuffer(), false))
+        {
+            CassandraStreamHeader header = CassandraStreamHeader.serializer.deserialize(in, MessagingService.current_version);
+            CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(messageHeader, header, session);
+            SSTableReader streamedSSTable = Iterables.getOnlyElement(reader.read(in).finished());
+
+            SSTableUtils.assertContentEquals(sstable, streamedSSTable);
+        }
+    }
+
+    private boolean indexSummaryRedistribution() throws IOException
+    {
+        long nonRedistributingOffHeapSize = 0;
+        long memoryPoolBytes = 1024 * 1024;
+
+        // rewrite index summary file with new min/max index interval
+        TableMetadata origin = store.metadata();
+        MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstable, OperationType.INDEX_SUMMARY))
+        {
+            IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                     nonRedistributingOffHeapSize,
+                                                                                     memoryPoolBytes));
+        }
+
+        // reset min/max index interval
+        MigrationManager.announceTableUpdate(origin, true);
+        return true;
+    }
+
+    private Future<?> executeAsync(Callable<?> mutation)

Review comment:
       +1

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")

Review comment:
       >  Then we'll suspend the summary saving from finishing while the write continues. 
   
   It does reproduce the issue more reliably when there is no lock.. With lock, the test will run into deadlock..

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {
+
+            Descriptor desc = sstable.descriptor;
+            desc.getMetadataSerializer().mutate(desc, "testing", stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+
+            return null;
+        });
+    }
+
+    @Test
+    @BMRule(name = "Delay saving index summary, manifest may link partially written file if there is no lock",
+            targetClass = "SSTableReader",
+            targetMethod = "saveSummary(Descriptor, DecoratedKey, DecoratedKey, IndexSummary)",
+            targetLocation = "AFTER INVOKE serialize",
+            condition = "$descriptor.cfname.contains(\"Standard1\")",
+            action = "org.apache.cassandra.db.streaming.EntireSSTableStreamConcurrentComponentMutationTest.countDown();Thread.sleep(5000);")

Review comment:
       >  Then we'll suspend the summary saving from finishing while the write continues. 
   
   It does reproduce the issue more reliably when there is no lock.. With lock, the test will run into deadlock because index summary didn't release lock, manifest cannot be created..

##########
File path: test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.streaming;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+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.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummaryManager;
+import org.apache.cassandra.io.sstable.IndexSummaryRedistribution;
+import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.net.BufferPoolAllocator;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.SharedDefaultFileRegion;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.OutgoingStream;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+import org.apache.cassandra.streaming.messages.StreamMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(BMUnitRunner.class)
+public class EntireSSTableStreamConcurrentComponentMutationTest
+{
+    public static final String KEYSPACE = "CassandraEntireSSTableStreamLockTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    private static final Callable<?> NO_OP = () -> null;
+
+    private static SSTableReader sstable;
+    private static Descriptor descriptor;
+    private static ColumnFamilyStore store;
+    private static RangesAtEndpoint rangesAtEndpoint;
+
+    private static ExecutorService service;
+
+    private static CountDownLatch latch = new CountDownLatch(1);
+
+    @BeforeClass
+    public static void defineSchemaAndPrepareSSTable()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingParams.CACHE_NOTHING));
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        store = keyspace.getColumnFamilyStore("Standard1");
+
+        // 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))
+            .clustering("0")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        sstable = store.getLiveSSTables().iterator().next();
+        descriptor = sstable.descriptor;
+
+        Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0));
+        Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100));
+        rangesAtEndpoint = RangesAtEndpoint.toDummyList(Collections.singleton(new Range<>(start, end)));
+
+        service = Executors.newFixedThreadPool(2);
+    }
+
+    @AfterClass
+    public static void cleanup()
+    {
+        service.shutdown();
+    }
+
+    @After
+    public void reset() throws IOException
+    {
+        latch = new CountDownLatch(1);
+        // reset repair info to avoid test interfering each other
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, ActiveRepairService.NO_PENDING_REPAIR, false);
+    }
+
+    @Test
+    public void testStream() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, NO_OP);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    @BMRule(name = "Let component mutation happen immediately after ComponentManifest completion",
+    targetClass = "ComponentManifest",
+    targetMethod = "<init>",
+    action = "Thread.sleep(3000)")
+    public void testStreamWithStatsMutation() throws Exception
+    {
+        testStreamWithConcurrentComponentMutation(NO_OP, () -> {

Review comment:
       +1, this will prove the need of on-demand manifest creation, rather than relying on the manifest created initially.

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -2067,6 +2081,21 @@ public int getSSTableLevel()
         return sstableMetadata.sstableLevel;
     }
 
+    /**
+     * Mutate Stats Metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
+     *
+     * @param description on changed attributes
+     * @param transform function to mutate stats metadata
+     */
+    public void mutateAndReloadStats(String description, UnaryOperator<StatsMetadata> transform) throws IOException

Review comment:
       good idea.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
dcapwell commented on pull request #642:
URL: https://github.com/apache/cassandra/pull/642#issuecomment-685069342


   > index summary redistribution and stats-mutation shouldn't happen together because both are running as part of compactions.
   
   You are right, I keep forgetting `org.apache.cassandra.db.lifecycle.Tracker#tryModify(java.lang.Iterable<org.apache.cassandra.io.sstable.format.SSTableReader>, org.apache.cassandra.db.compaction.OperationType)` marks sstables as compacting, so compactions will avoid compacting those files
   
   ```
   if (null == apply(permitCompacting(sstables), updateCompacting(emptySet(), sstables)))
               return null;
           return new LifecycleTransaction(this, operationType, sstables);
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r473657580



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
##########
@@ -103,9 +104,11 @@ public SSTableWriter open(Descriptor descriptor,
     static class ReaderFactory extends SSTableReader.Factory
     {
         @Override
-        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata,
+                                  Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason,
+                                  SerializationHeader header, IndexSummary summary, FileHandle dfile, FileHandle ifile, IFilter bf)

Review comment:
       I extract `Builder` into a separate class and pass it as the only argument to `open`.. WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r481365485



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,38 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;

Review comment:
       the API allows it to be other things, so would still require this check to avoid future changes causing class cast issues.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r468524097



##########
File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
##########
@@ -154,29 +143,43 @@ public UUID getPendingRepair()
         return ref.get().getPendingRepair();
     }
 
-    public int getManifestSize()
-    {
-        return manifest.components().size();
-    }
-
     @Override
     public void write(StreamSession session, DataOutputStreamPlus out, int version) throws IOException
     {
+        boolean streamEntirely = shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus;
         SSTableReader sstable = ref.get();
-        CassandraStreamHeader.serializer.serialize(header, out, version);
-        out.flush();
 
-        if (shouldStreamEntireSSTable && out instanceof AsyncStreamingOutputPlus)
+        if (streamEntirely)
         {
-            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
-            writer.write((AsyncStreamingOutputPlus) out);
+            // Acquire lock to avoid concurrent sstable component mutation becaue of stats update or index summary
+            // redistribution, otherwise file sizes recorded in component manifest will be different from actual
+            // file sizes. (Note: Windows doesn't support atomic replace and index summary redistribution deletes
+            // existing file first)
+            // Recreate the latest manifest with hard links in case components are modified.
+            ComponentManifest manifest = sstable.runWithReadLock(ignored -> ComponentManifest.create(sstable, true));

Review comment:
       +1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469469409



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentContext.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.streaming;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Mutable SSTable components and their hardlinks to avoid concurrent sstable component modification
+ * during entire-sstable-streaming.
+ */
+public class ComponentContext implements AutoCloseable
+{
+    private static final Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(Component.STATS, Component.SUMMARY);
+
+    private final Map<Component, File> hardLinks;
+    private final ComponentManifest manifest;
+
+    private ComponentContext(Map<Component, File> hardLinks, ComponentManifest manifest)
+    {
+        this.hardLinks = hardLinks;
+        this.manifest = manifest;
+    }
+
+    public static ComponentContext create(Descriptor descriptor)
+    {
+        LinkedHashMap<Component, File> hardLinks = new LinkedHashMap<>(1);
+
+        for (Component component : MUTABLE_COMPONENTS)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            if (!file.exists())
+                continue;
+
+            File hardlink = new File(descriptor.tmpFilenameForStreaming(component));
+            FileUtils.createHardLink(file, hardlink);
+            hardLinks.put(component, hardlink);
+        }
+
+        return new ComponentContext(hardLinks, ComponentManifest.create(descriptor));
+    }
+
+    public ComponentManifest manifest()
+    {
+        return manifest;
+    }
+
+    /**
+     * @return file channel to be streamed, either original component or hardlinked component.
+     */
+    @SuppressWarnings("resource") // file channel will be closed by Caller

Review comment:
       @jasonstack I think what we actually want is a `@SuppressWarnings("IOResourceOpenedButNotSafelyClosed")` on the line where we create the `RandomAccessFile`. (Then we can remove this one.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469940059



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I would have to spend a while familiarising myself with this patch (and all present usages) to be sure, and all I've done to answer is skim the existing 3.0 SSTableReader, but this is probably fine. Any plain write that occurs before the SSTable is published will piggy-back on the `volatile` write used to publish the SSTable to other threads. I think in any cases we care about it _should_ piggy-back on `Tidy.setup` before this.  
   
   The only likely issue, then, is if we were to modify the summary while the sstable is already published.  Even in this case we're fine because `IndexSummary` is an immutable class (by which I mean its fields are all final), so the worst that can happen is that it is not immediately visible (technically the JMM doesn't guarantee it is ever visible to other threads, but in practice it cannot fail to eventually make it visible)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r459634173



##########
File path: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java
##########
@@ -30,16 +35,31 @@
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.AsyncStreamingOutputPlus;
+import org.apache.cassandra.streaming.StreamManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static java.lang.Math.min;
 
 public final class ComponentManifest implements Iterable<Component>

Review comment:
       We could almost call this thing `ComponentStreamer` now :)
   
   (I mean even if we go the hard-link + atomic move direction, it could be nice to delegate that here, given we'll have a spec per component that includes either the hard link or the canonical file to stream.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic closed pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #642:
URL: https://github.com/apache/cassandra/pull/642


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r458434647



##########
File path: test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java
##########
@@ -126,43 +152,98 @@ public void testBlockWriterOverWire() throws IOException
     @Test
     public void testBlockReadingAndWritingOverWire() throws Exception
     {
-        StreamSession session = setupStreamingSessionForTest();
-        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        testBlockReadingAndWritingOverWire(null);
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent stats metadata
+     * update causes the actual transfered file size to be different from the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithStatsMutation() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+            sstable.mutateAndReloadStats(stats -> stats.mutateRepairedMetadata(0, UUID.randomUUID(), false));
+            return null;
+        });
+    }
+
+    /**
+     * Entire-sstable-streaming receiver will throw checksum validation failure because concurrent index summary
+     * redistribution task will modify index summary causing actual transfered index summary size to be different from
+     * the one in {@link ComponentManifest}
+     */
+    @Test
+    public void testBlockReadingAndWritingOverWireWithIndexSummaryRedistribution() throws Exception
+    {
+        testBlockReadingAndWritingOverWire(() -> {
+
+            long nonRedistributingOffHeapSize = 0;
+            long memoryPoolBytes = 1024 * 1024;
+
+            // rewrite index summary file with new min/max index interval
+            TableMetadata origin = store.metadata();
+            MigrationManager.announceTableUpdate(origin.unbuild().minIndexInterval(1).maxIndexInterval(2).build(), true);
+
+            try (LifecycleTransaction txn = store.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+            {
+                IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(ImmutableMap.of(store.metadata().id, txn),
+                                                                                         nonRedistributingOffHeapSize,
+                                                                                         memoryPoolBytes));
+            }
+            descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, UUID.randomUUID(), false);
 
-        CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable));
+            // reset min/max index interval
+            MigrationManager.announceTableUpdate(origin, true);
 
+            return null;
+        });
+    }
+
+    public void testBlockReadingAndWritingOverWire(Callable<?> componentMutation) throws Exception
+    {
         // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed
         ByteBuf serializedFile = Unpooled.buffer(8192);
-        EmbeddedChannel channel = createMockNettyChannel(serializedFile);
-        AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+        InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort();
+        StreamSession session = setupStreamingSessionForTest();
+        ComponentManifest manifest = CassandraOutgoingFile.getComponentManifest(sstable);
+        CassandraStreamHeader header = header(manifest);
 
-        writer.write(out);
+        Future<?> future = sstable.runWithReadLock(ignored -> {
+            CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest);
+
+            // start concurrent sstable component mutation
+            Future<?> mutation = CompletableFuture.completedFuture(null);
+            if (componentMutation != null)
+            {
+                mutation = ScheduledExecutors.optionalTasks.submit(componentMutation);
+                // wait 1s for concurrent task to start
+                FBUtilities.sleepQuietly(1000);
+            }
+
+            EmbeddedChannel channel = createMockNettyChannel(serializedFile);
+            AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel);
+
+            writer.write(out);
+            return mutation;
+        });
+
+        // wait for concurrent component mutation to finish
+        future.get(15, TimeUnit.SECONDS);
 
         session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104));
 
-        CassandraStreamHeader header =
-            CassandraStreamHeader.builder()
-                                 .withSSTableFormat(sstable.descriptor.formatType)
-                                 .withSSTableVersion(sstable.descriptor.version)
-                                 .withSSTableLevel(0)
-                                 .withEstimatedKeys(sstable.estimatedKeys())
-                                 .withSections(Collections.emptyList())
-                                 .withSerializationHeader(sstable.header.toComponent())
-                                 .withComponentManifest(CassandraOutgoingFile.getComponentManifest(sstable))
-                                 .isEntireSSTable(true)
-                                 .withFirstKey(sstable.first)
-                                 .withTableId(sstable.metadata().id)
-                                 .build();
-
-        CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null), header, session);
+        StreamMessageHeader messageHeader = new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), false, 0, 0, 0, null);
+        CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(messageHeader, header, session);
 
         SSTableMultiWriter sstableWriter = reader.read(new DataInputBuffer(serializedFile.nioBuffer(), false));

Review comment:
       nit: Should we instantiate `sstableWriter` at the start of a try-with-resources block?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469588637



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       @bdeggleston @jasonstack I [mentioned](https://issues.apache.org/jira/browse/CASSANDRA-15861?focusedCommentId=17170484&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17170484) this briefly in the Jira, but with `indexSummary` not `final` or `volatile` in `SSTableReader`, do we have a visibility issue? The thread that loads/builds the summary needs to safely publish its changes for the thread that builds new summaries here.
   
   Simply making `indexSummary` `volatile` might be acceptable. Most of its accesses are themselves part of the summary redistribution task itself or during compaction, so we probably don't care about that overhead. `SSTableReader#getIndexScanPosition()`, on the other hand, does look like it's used at least during some range reads.
   
   We could go back to the old scope for the `synchronized` block, leaving `indexSummary` non-volatile. That would at least ensure visibility between the `cloneX()` methods. Even then, there are other things that call `SSTableReader#internalOpen()`, like `BigTableWriter.openEarly()` and `openFinal()`, that don't initialize `indexSummary` while holding the monitor lock. (Perhaps we could just synchronize to get the `IndexSummary` reference, rather than having to wait for `buildSummaryAtLevel()`, though.)
   
   Is there actually a bug here independent of this patch? i.e. Do we actually need to make sure the block that includes setting `indexSummary` in `SSTableReader#internalOpen()` is performed in a `synchronized (tidy.global)` block? This bring up another level of puzzlement for me, which is why we ever synchronize on the non-final `global` field, rather than just `tidy`, which seems like it would actually make it safe to do what I've mentioned above.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469940059



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       I would have to spend a while familiarising myself with this patch (and all present usages) to be sure, but this is probably fine - the plain write will piggy-back on whatever `volatile` write is used to publish the SSTable to other threads. I think in any cases we care about it _should_ piggy-back on `Tidy.setup`.  The only possible issue is if we modify the summary while the sstable is already published, and even in this case we're fine because `IndexSummary` is an immutable class (by which I mean its fields are all final), so the worst that can happen is that it is not immediately visible (technically the JMM doesn't guarantee it is ever visible to other threads, but in practice it cannot fail to eventually make it visible)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r469474266



##########
File path: test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java
##########
@@ -40,12 +40,7 @@
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader;
-import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamWriter;
-import org.apache.cassandra.db.streaming.CassandraOutgoingFile;
-import org.apache.cassandra.db.streaming.CassandraStreamHeader;
-import org.apache.cassandra.db.streaming.CassandraStreamReader;
-import org.apache.cassandra.db.streaming.CassandraStreamWriter;
+import org.apache.cassandra.db.streaming.*;

Review comment:
       nit: I think we generally try to avoid wildcards.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a change in pull request #642: Cassandra 15861 lock trunk

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on a change in pull request #642:
URL: https://github.com/apache/cassandra/pull/642#discussion_r470038096



##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       So, for the result of `cloneWithNewSummaryAtLevel` to ever be visible to anybody, it must be published safely - in this case ordinarily through the DataTracker.View. So anything we do to provide visibility earlier than this is just providing is some nicer feels, which is fine, but as it happens we have several such volatile writes: the synchronised block when saving the new summary provides visibility, as does the write to `tidy.setup` (which is literally its purpose, iirc, though for the scope of `tidy`).
   
   I am generally in favour of making more explicit links between a visibility requirement and where it is enforced, but there's definitely no need to introduce extra synchronised blocks - a simple write to a volatile property is sufficient, and in this case a comment indicating we're piggy-backing on any of these mentioned volatile writes might be sufficient - as would none of them, since we must publish the sstable safely anyway.  Stylistically, it's all up to you guys of course.

##########
File path: src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
##########
@@ -1202,39 +1215,38 @@ public void run()
     @SuppressWarnings("resource")
     public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
     {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
+        assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata().params.minIndexInterval;
-            int maxIndexInterval = metadata().params.maxIndexInterval;
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+        int minIndexInterval = metadata().params.minIndexInterval;
+        int maxIndexInterval = metadata().params.maxIndexInterval;
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();

Review comment:
       (This class is definitely a mess)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org