Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-9295

Streaming not holding on to refs long enough.

    XMLWordPrintableJSON

Details

    • Normal

    Description

      While doing some testing around adding/removing nodes under load with cassandra-2.1 head as of a few days ago (after was 2.1.5 tagged) I am seeing stream out errors with file not found exceptions. The file in question just finished being compacted into a new file a few lines earlier in the log. Seems that streaming isn't holding onto Ref's correctly for the stuff in the stream plans.

      I also see a "corrupt sstable" exception for the file the "missing" file was compacted to. Trimmed logs with just the compaction/streaming related stuff:
      You can see the stream plan is initiated in between the compaction starting, and the compaction finishing.

      INFO  [MemtableFlushWriter:3] 2015-05-04 16:08:21,239  Memtable.java:380 - Completed flushing /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db (60666088 bytes) for commitlog position ReplayPosition(segmentId=1430755416941, position=32294797)
      INFO  [CompactionExecutor:4] 2015-05-04 16:08:40,856  CompactionTask.java:140 - Compacting [SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db'), SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-3-Data.db')]
      INFO  [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,047  StreamResultFuture.java:109 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Creating new streaming plan for Rebuild
      INFO  [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,238  StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild
      INFO  [STREAM-INIT-/10.240.213.56:53192] 2015-05-04 16:09:31,249  StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild
      INFO  [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:31,353  ColumnFamilyStore.java:882 - Enqueuing flush of standard1: 91768068 (19%) on-heap, 0 (0%) off-heap
      INFO  [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:37,425  ColumnFamilyStore.java:882 - Enqueuing flush of solr: 10012689 (2%) on-heap, 0 (0%) off-heap
      INFO  [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:38,073  StreamResultFuture.java:166 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Prepare completed. Receiving 0 files(0 bytes), sending 6 files(284288285 bytes)
      INFO  [CompactionExecutor:4] 2015-05-04 16:10:11,047  CompactionTask.java:270 - Compacted 2 sstables to [/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5,/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-8,].  182,162,816 bytes to 182,162,816 (~100% of original) in 90,188ms = 1.926243MB/s.  339,856 total partitions merged to 339,856.  Partition merge counts were {1:339856, }
      ERROR [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,169  StreamSession.java:477 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Streaming error occurred
      java.io.IOException: Corrupted SSTable : /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5-Data.db
      	at org.apache.cassandra.io.util.DataIntegrityMetadata$ChecksumValidator.validate(DataIntegrityMetadata.java:79) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:149) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:102) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:58) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:346) [cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:318) [cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_40]
      INFO  [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,232  StreamResultFuture.java:180 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Session with /10.240.213.56 is complete
      WARN  [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,269  StreamResultFuture.java:207 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Stream failed
      ERROR [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,307  StreamSession.java:477 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Streaming error occurred
      java.lang.RuntimeException: java.io.FileNotFoundException: /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db (No such file or directory)
      	at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:124) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:107) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.io.util.SegmentedFile.createReader(SegmentedFile.java:99) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1955) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:74) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:58) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:346) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:326) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_40]
      Caused by: java.io.FileNotFoundException: /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db (No such file or directory)
      	at java.io.RandomAccessFile.open0(Native Method) ~[na:1.8.0_40]
      	at java.io.RandomAccessFile.open(RandomAccessFile.java:316) ~[na:1.8.0_40]
      	at java.io.RandomAccessFile.<init>(RandomAccessFile.java:243) ~[na:1.8.0_40]
      	at org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:64) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	at org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:120) ~[cassandra-all-2.1.5.426.jar:2.1.5.426]
      	... 10 common frames omitted
      

      Attachments

        1. 9295.debug.txt
          10 kB
          Benedict Elliott Smith
        2. 0001-don-t-release-sstable-while-streaming.patch
          14 kB
          Yuki Morishita
        3. 0002-demonstrate-checksum-failure-of-early-opened-SSTable.patch
          6 kB
          Yuki Morishita
        4. 0003-Do-not-stream-early-opened-SSTable.patch
          5 kB
          Yuki Morishita

        Activity

          People

            yukim Yuki Morishita
            jjordan Jeremiah Jordan
            Yuki Morishita
            Benedict Elliott Smith
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: