[ 
https://issues.apache.org/jira/browse/CASSANDRA-5381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13615171#comment-13615171
 ] 

Mathijs Vogelzang commented on CASSANDRA-5381:
----------------------------------------------

We have the same issue where all streaming between nodes fails with an 
EOFException and then "too many retries". This started when we upgraded from 
1.1.7 to 1.2.2, and didn't go away on subsequent upgrade to 1.2.3.

We tried running with/without internode compression and encryption, and found 
out that when encryption is off, everything works fine (also WITH compression 
on). With encryption on, it doesn't work, also with internode compression 
turned off, so for us it definitely has something to do with streaming while 
internode encryption is enabled.
                
> java.io.EOFException exception while executing nodetool repair with 
> compression enabled
> ---------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-5381
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-5381
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.2.3
>         Environment: Linux Virtual Machines, Red Hat Enterprise release 6.4, 
> kernel version  2.6.32-358.2.1.el6.x86_64. Each VM has 8GB memory and 4vCPUS.
>            Reporter: Neil Thomson
>            Priority: Minor
>
> Very similar to issue reported in CASSANDRA-5105. I have 3 nodes configured 
> in a cluster. The nodes are configured with compression enabled. When 
> attempting a nodetool repair on one node, i get exceptions in the other nodes 
> in the cluster.
> Disabling compression on the column family allows nodetool repair to run 
> without error.
> Exception:
> INFO [Streaming to /3.69.211.179:2] 2013-03-25 12:30:27,874 
> StreamReplyVerbHandler.java (line 50) Need to re-stream file 
> /var/lib/cassandra/data/rt/values/rt-values-ib-1-Data.db to /3.69.211.179
> INFO [Streaming to /3.69.211.179:2] 2013-03-25 12:30:27,991 
> StreamReplyVerbHandler.java (line 50) Need to re-stream file 
> /var/lib/cassandra/data/rt/values/rt-values-ib-1-Data.db to /3.69.211.179
> ERROR [Streaming to /3.69.211.179:2] 2013-03-25 12:30:28,113 
> CassandraDaemon.java (line 164) Exception in thread Thread[Streaming to 
> /3.69.211.179:2,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
> Keyspace configuration is as follows:
> Keyspace: rt:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
>     Options: [replication_factor:3]
>   Column Families:
>     ColumnFamily: tagname
>       Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>       Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>       Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>       GC grace seconds: 864000
>       Compaction min/max thresholds: 4/32
>       Read repair chance: 0.1
>       DC Local Read repair chance: 0.0
>       Populate IO Cache on flush: false
>       Replicate on write: true
>       Caching: KEYS_ONLY
>       Bloom Filter FP chance: default
>       Built indexes: []
>       Compaction Strategy: 
> org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy
>     ColumnFamily: values
>       Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>       Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>       Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>       GC grace seconds: 864000
>       Compaction min/max thresholds: 4/32
>       Read repair chance: 0.1
>       DC Local Read repair chance: 0.0
>       Populate IO Cache on flush: false
>       Replicate on write: true
>       Caching: KEYS_ONLY
>       Bloom Filter FP chance: default
>       Built indexes: []
>       Compaction Strategy: 
> org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to