[jira] [Commented] (CASSANDRA-10008) Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)

2015-08-07 Thread Chris Moos (JIRA)

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

Chris Moos commented on CASSANDRA-10008:


FYI Tested this patch on my node experiencing the upgrade issue and it was able 
to successfully upgrade the SSTables.

> Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)
> 
>
> Key: CASSANDRA-10008
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10008
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Chris Moos
>Assignee: Chris Moos
> Fix For: 2.2.x
>
> Attachments: CASSANDRA-10008.patch
>
>
> Running *nodetool upgradesstables* fails with the following after upgrading 
> to 2.2.0 from 2.1.2:
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10008) Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10008:
---
Attachment: CASSANDRA-10008.patch

> Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)
> 
>
> Key: CASSANDRA-10008
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10008
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Chris Moos
> Fix For: 2.2.x
>
> Attachments: CASSANDRA-10008.patch
>
>
> Running *nodetool upgradesstables* fails with the following after upgrading 
> to 2.2.0 from 2.1.2:
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-10008) Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)

2015-08-06 Thread Chris Moos (JIRA)

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

Chris Moos edited comment on CASSANDRA-10008 at 8/7/15 6:46 AM:


I think I may have tracked down the issue, it seems that this happens because 
sstables from the old version and new version both exist in the transaction and 
during upgrade cancel() is used to filter out the latest versions sstables, but 
cancel() is not fully removing the SSTableReader from the transaction so 
checkUnused() fails.

Patch attached.


was (Author: chrismoos):
I think I may have tracked down the issue, it seems that this happens because 
sstables from the old version and new version both exist in the transaction and 
during upgrade cancel() is used to filter out the latest versions sstables, but 
cancel() is not fully removing the SSTableReader from the transaction.

Patch attached.

> Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)
> 
>
> Key: CASSANDRA-10008
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10008
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Chris Moos
> Fix For: 2.2.x
>
>
> Running *nodetool upgradesstables* fails with the following after upgrading 
> to 2.2.0 from 2.1.2:
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10008) Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)

2015-08-06 Thread Chris Moos (JIRA)

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

Chris Moos commented on CASSANDRA-10008:


I think I may have tracked down the issue, it seems that this happens because 
sstables from the old version and new version both exist in the transaction and 
during upgrade cancel() is used to filter out the latest versions sstables, but 
cancel() is not fully removing the SSTableReader from the transaction.

Patch attached.

> Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)
> 
>
> Key: CASSANDRA-10008
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10008
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Chris Moos
> Fix For: 2.2.x
>
>
> Running *nodetool upgradesstables* fails with the following after upgrading 
> to 2.2.0 from 2.1.2:
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10012) Deadlock when session streaming is retried after exception

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10012?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10012:
---
Attachment: CASSANDRA-10012.patch

> Deadlock when session streaming is retried after exception
> --
>
> Key: CASSANDRA-10012
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10012
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Chris Moos
> Attachments: CASSANDRA-10012.patch
>
>
> This patch ensures that the CompressedInputStream thread is cleaned up 
> properly (for example, if an Exception occurs and a session stream is 
> retried).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10005) Streaming not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

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

Chris Moos commented on CASSANDRA-10005:


new issue created for this, patch attached:

CASSANDRA-10012

> Streaming not enough bytes error
> 
>
> Key: CASSANDRA-10005
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10005
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Chris Moos
>Priority: Minor
> Fix For: 2.2.x
>
> Attachments: deadlock.txt, errors.txt
>
>
> I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
> below and the node never joins. It looks like a deadlock.
> After looking through the code it looks like IncomingFileMessage will tell 
> the session to retry on Exceptions (except IOException) but the 
> CompressedInputStream thread is still running and then the retry happens and 
> the deadlock ensues. It might be best to close the StreamReader (and stop the 
> thread) if an Exception happens before retrying.
> I'm not sure why I am getting this error to begin with though, might it have 
> something to do with not being able to upgrade my SSTables after going from 
> 2.1.2 -> 2.2.0?
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-10012) Deadlock when session streaming is retried after exception

2015-08-06 Thread Chris Moos (JIRA)
Chris Moos created CASSANDRA-10012:
--

 Summary: Deadlock when session streaming is retried after exception
 Key: CASSANDRA-10012
 URL: https://issues.apache.org/jira/browse/CASSANDRA-10012
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Chris Moos


This patch ensures that the CompressedInputStream thread is cleaned up properly 
(for example, if an Exception occurs and a session stream is retried).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-10008) Upgrading SSTables fails on 2.2.0 (after upgrade from 2.1.2)

2015-08-06 Thread Chris Moos (JIRA)
Chris Moos created CASSANDRA-10008:
--

 Summary: Upgrading SSTables fails on 2.2.0 (after upgrade from 
2.1.2)
 Key: CASSANDRA-10008
 URL: https://issues.apache.org/jira/browse/CASSANDRA-10008
 Project: Cassandra
  Issue Type: Bug
Reporter: Chris Moos


Running *nodetool upgradesstables* fails with the following after upgrading to 
2.2.0 from 2.1.2:

{code}
error: null
-- StackTrace --
java.lang.AssertionError
at 
org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
at 
org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
at 
org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
at 
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
at 
org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
at 
org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10005) Streaming not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10005:
---
Description: 
I'm adding a new node to the cluster and I'm seeing a bunch of the errors below 
and the node never joins. It looks like a deadlock.

After looking through the code it looks like IncomingFileMessage will tell the 
session to retry on Exceptions (except IOException) but the 
CompressedInputStream thread is still running and then the retry happens and 
the deadlock ensues. It might be best to close the StreamReader (and stop the 
thread) if an Exception happens before retrying.

I'm not sure why I am getting this error to begin with though, might it have 
something to do with not being able to upgrade my SSTables after going from 
2.1.2 -> 2.2.0?

{code}
error: null
-- StackTrace --
java.lang.AssertionError
at 
org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
at 
org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
at 
org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
at 
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
at 
org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
at 
org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
{code}

  was:
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. This causes a deadlock, (see deadlock 
below):

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
 ~[guava-16.0.jar:na]
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:162) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 [apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

{code}
ERROR 06:28:26 [Stream #059b7cc0-3c04-11e5-8c56-dbbae7f19873] Streaming error 
occurred
java.lang.IllegalArgumentException: Unknown type 0
at 
org.apache.cassandra.streaming.messages.StreamMessage$Type.get(StreamMessage.java:90)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

Found one Java-level deadlock:
=
"Thread-869":
  wait

[jira] [Updated] (CASSANDRA-10005) Streaming not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10005:
---
Attachment: errors.txt
deadlock.txt

> Streaming not enough bytes error
> 
>
> Key: CASSANDRA-10005
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10005
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Chris Moos
>Priority: Minor
> Attachments: deadlock.txt, errors.txt
>
>
> I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
> below and the node never joins. It looks like a deadlock.
> After looking through the code it looks like IncomingFileMessage will tell 
> the session to retry on Exceptions (except IOException) but the 
> CompressedInputStream thread is still running and then the retry happens and 
> the deadlock ensues. It might be best to close the StreamReader (and stop the 
> thread) if an Exception happens before retrying.
> I'm not sure why I am getting this error to begin with though, might it have 
> something to do with not being able to upgrade my SSTables after going from 
> 2.1.2 -> 2.2.0?
> {code}
> error: null
> -- StackTrace --
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkUnused(LifecycleTransaction.java:428)
> at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.split(LifecycleTransaction.java:408)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:268)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:373)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1524)
> at 
> org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:2521)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10005) Streaming not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10005:
---
Description: 
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. This causes a deadlock, (see deadlock 
below):

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
 ~[guava-16.0.jar:na]
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:162) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 [apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

{code}
ERROR 06:28:26 [Stream #059b7cc0-3c04-11e5-8c56-dbbae7f19873] Streaming error 
occurred
java.lang.IllegalArgumentException: Unknown type 0
at 
org.apache.cassandra.streaming.messages.StreamMessage$Type.get(StreamMessage.java:90)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

Found one Java-level deadlock:
=
"Thread-869":
  waiting to lock monitor 0x7f2ef8003f08 (object 0x00062b8a9a28, a 
java.lang.Object),
  which is held by "STREAM-IN-/10.220.0.147"
"STREAM-IN-/10.220.0.147":
  waiting to lock monitor 0x7f2ed00436a8 (object 0x00062bc96d68, a 
java.lang.Object),
  which is held by "Thread-869"

Java stack information for the threads listed above:
===
"Thread-869":
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:295)
- waiting to lock <0x00062b8a9a28> (a java.lang.Object)
at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:59)
- locked <0x00062bc96d68> (a java.lang.Object)
at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
- locked <0x0006340e3f30> (a sun.nio.ch.ChannelInputStream)
at 
org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:161)
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at java.lang.Thread.run(Thread.java:745)
"STREAM-IN-/10.220.0.147":
at 
java.nio.channels.spi.AbstractSelectableChannel.isBlocking(AbstractSelectableChannel.java:261)
- waiting to lock <0x00062bc96d68> (a java.lang.Object)
at su

[jira] [Updated] (CASSANDRA-10005) Streaming not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10005:
---
Description: 
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. Does anyone know anything about this 
error? I don't know if its related to why its stalling..

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
 ~[guava-16.0.jar:na]
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:162) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 [apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

{code}
ERROR 06:28:26 [Stream #059b7cc0-3c04-11e5-8c56-dbbae7f19873] Streaming error 
occurred
java.lang.IllegalArgumentException: Unknown type 0
at 
org.apache.cassandra.streaming.messages.StreamMessage$Type.get(StreamMessage.java:90)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

  was:
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. Does anyone know anything about this 
error?

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIt

[jira] [Updated] (CASSANDRA-10005) Streaming fails with not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chris Moos updated CASSANDRA-10005:
---
Description: 
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. Does anyone know anything about this 
error?

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
 ~[guava-16.0.jar:na]
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:162) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 [apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

{code}
ERROR 06:28:26 [Stream #059b7cc0-3c04-11e5-8c56-dbbae7f19873] Streaming error 
occurred
java.lang.IllegalArgumentException: Unknown type 0
at 
org.apache.cassandra.streaming.messages.StreamMessage$Type.get(StreamMessage.java:90)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}

  was:
I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. Does anyone know anything about this 
error?

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)

[jira] [Created] (CASSANDRA-10005) Streaming fails with not enough bytes error

2015-08-06 Thread Chris Moos (JIRA)
Chris Moos created CASSANDRA-10005:
--

 Summary: Streaming fails with not enough bytes error
 Key: CASSANDRA-10005
 URL: https://issues.apache.org/jira/browse/CASSANDRA-10005
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Chris Moos
Priority: Minor


I'm adding a new node to the cluster and I'm seeing a bunch of the errors 
below. The node never joins the cluster. Does anyone know anything about this 
error?

{code}
WARN  [STREAM-IN-/10.220.0.160] 2015-08-06 16:16:42,640 StreamSession.java:638 
- [Stream #4be6d7c0-3c53-11e5-b5bc-dbbae7f19873] Retrying for following error
java.lang.IllegalArgumentException: Not enough bytes
at 
org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
 ~[guava-16.0.jar:na]
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:162) 
~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
 ~[apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
 [apache-cassandra-2.2.0.jar:2.2.0]
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 [apache-cassandra-2.2.0.jar:2.2.0]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_79]
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)