XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Urgent
    • Resolution: Fixed
    • 1.2.5
    • None
    • None
    • 5 nodes, vnodes enabled, encryption disabled, compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA enabled.

    • Critical

    Description

      When I run nodetool repair on cas01 node it get's stuck at some point.

      I see following exceptions in cas01 system.log:

      ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to /10.10.45.60:28,5,main]
      java.lang.RuntimeException: java.io.EOFException
      at com.google.common.base.Throwables.propagate(Throwables.java:160)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
      at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
      at java.lang.Thread.run(Unknown Source)
      Caused by: java.io.EOFException
      at java.io.DataInputStream.readInt(Unknown Source)
      at org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
      at org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
      at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      ... 3 more

      ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) Exception in thread Thread[Thread-2076,5,main]
      java.lang.AssertionError: incorrect row data size 130921 written to /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db; correct is 131074
      at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
      at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
      at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
      at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
      at org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
      at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)

      On other machines there are some exceptions too:

      ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) Exception in thread Thread[Thread-1424,5,main]
      java.lang.AssertionError: incorrect row data size 130921 written to /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db; correct is 131074
      at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
      at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
      at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
      at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
      at org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
      at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
      ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to /10.10.45.58:55,5,main]
      java.lang.RuntimeException: java.io.EOFException
      at com.google.common.base.Throwables.propagate(Throwables.java:160)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
      at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
      at java.lang.Thread.run(Unknown Source)
      Caused by: java.io.EOFException
      at java.io.DataInputStream.readInt(Unknown Source)
      at org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
      at org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
      at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      ... 3 more

      Then I see frozen status in nodetool netstats and repair never completes.

      Attachments

        1. 0001-add-RangeTombstone-transfer-test.patch
          3 kB
          Yuki Morishita
        2. 5418-1.2.txt
          1 kB
          Igor Ivanov
        3. 5418-1.2-v2.txt
          4 kB
          Igor Ivanov
        4. 5418-1.2-v3.txt
          4 kB
          Igor Ivanov
        5. 5418-v4.txt
          4 kB
          Sylvain Lebresne

        Activity

          People

            slebresne Sylvain Lebresne
            radev Igor Ivanov
            Sylvain Lebresne
            Yuki Morishita
            Votes:
            1 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: