[
https://issues.apache.org/jira/browse/CASSANDRA-5391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13616307#comment-13616307
]
Ondřej Černoš commented on CASSANDRA-5391:
------------------------------------------
I am becoming quite sure the problem is a race condition in Cassandra code
handling decompression of sstables when these are streamed from the remote
datacenter.
Both traces - when snappy is used and when the java zip is used - share the
same calls, see above.
I switched trace level in log4j and this is what I found:
* when 2 and more nodes live in the remote DC, cassandra fires two threads
downloading the same file
* when only 1 node lives in the remote DC, only one thread downloads the file
This is how it looks in log:
{noformat}
2013-03-28 13:44:57.301+0100 [Thread-22] [DEBUG] StreamInSession.java(104)
org.apache.cassandra.streaming.StreamInSession: Adding file
/path/to/cassandra/data/ks/cf/ks-cf-ib-2-Data.db to Stream Request queue
2013-03-28 13:44:57.301+0100 [Thread-22] [DEBUG] StreamInSession.java(104)
org.apache.cassandra.streaming.StreamInSession: Adding file
/path/to/cassandra/data/ks/cf/ks-cf-ib-1-Data.db to Stream Request queue
2013-03-28 13:44:57.338+0100 [Thread-23] [DEBUG] StreamInSession.java(104)
org.apache.cassandra.streaming.StreamInSession: Adding file
/path/to/cassandra/data/ks/cf/ks-cf-ib-2-Data.db to Stream Request queue
2013-03-28 13:44:57.340+0100 [Thread-23] [DEBUG] StreamInSession.java(104)
org.apache.cassandra.streaming.StreamInSession: Adding file
/path/to/cassandra/data/ks/cf/ks-cf-ib-1-Data.db to Stream Request queue
{noformat}
And here is the result grepped on the two threads:
{noformat}
2013-03-28 13:44:57.477+0100 [Thread-22] [TRACE] SSTableWriter.java(145)
org.apache.cassandra.io.sstable.SSTableWriter: wrote
DecoratedKey(-8516046549581000893,
6663363133663230623932663663303732623735653332643964616261623165) at 183591
2013-03-28 13:44:57.477+0100 [Thread-22] [TRACE] SSTableWriter.java(463)
org.apache.cassandra.io.sstable.SSTableWriter: wrote index entry:
org.apache.cassandra.db.RowIndexEntry@7b553d18 at 16192
2013-03-28 13:44:57.477+0100 [Thread-22] [TRACE] SSTableWriter.java(145)
org.apache.cassandra.io.sstable.SSTableWriter: wrote
DecoratedKey(-8513551951874950453,
3934363831326161323235653165613662613039346233356264386461653735) at 183995
2013-03-28 13:44:57.478+0100 [Thread-22] [TRACE] SSTableWriter.java(463)
org.apache.cassandra.io.sstable.SSTableWriter: wrote index entry:
org.apache.cassandra.db.RowIndexEntry@d5f0688 at 16238
2013-03-28 13:44:57.501+0100 [Thread-22] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-1-Data.db
2013-03-28 13:44:57.501+0100 [Thread-22] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-1-Filter.db
2013-03-28 13:44:57.501+0100 [Thread-22] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-1-TOC.txt
2013-03-28 13:44:57.501+0100 [Thread-22] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting
ks-cf-tmp-ib-1-CompressionInfo.db
2013-03-28 13:44:57.502+0100 [Thread-22] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-1-Index.db
2013-03-28 13:44:57.502+0100 [Thread-22] [DEBUG] SSTable.java(154)
org.apache.cassandra.io.sstable.SSTable: Deleted
/path/to/cassandra/data/ks/cf/ks-cf-tmp-ib-1
2013-03-28 13:44:57.503+0100 [Thread-22] [INFO] StreamInSession.java(136)
org.apache.cassandra.streaming.StreamInSession: Streaming of file
/path/to/cassandra/data/ks/cf/ks-cf-ib-2-Data.db sections=130
progress=67628/1583497 - 4% for
org.apache.cassandra.streaming.StreamInSession@21400eb0 failed: requesting a
retry.
2013-03-28 13:44:57.504+0100 [Thread-22] [DEBUG] IncomingTcpConnection.java(91)
org.apache.cassandra.net.IncomingTcpConnection: IOException reading from
socket; closing
java.io.IOException: CRC unmatched
at
org.apache.cassandra.streaming.compress.CompressedInputStream.decompress(CompressedInputStream.java:111)
at
org.apache.cassandra.streaming.compress.CompressedInputStream.read(CompressedInputStream.java:79)
at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:320)
at
org.apache.cassandra.utils.BytesReadTracker.readUnsignedShort(BytesReadTracker.java:140)
at
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
at
org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:160)
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)
2013-03-28 13:44:58.070+0100 [Thread-23] [TRACE] SSTableWriter.java(463)
org.apache.cassandra.io.sstable.SSTableWriter: wrote index entry:
org.apache.cassandra.db.RowIndexEntry@db766c1 at 106582
2013-03-28 13:44:58.071+0100 [Thread-23] [TRACE] SSTableWriter.java(145)
org.apache.cassandra.io.sstable.SSTableWriter: wrote
DecoratedKey(-4829320003365722996,
6333383266393230353964313666633136356335333437353637373735653065) at 1217942
2013-03-28 13:44:58.071+0100 [Thread-23] [TRACE] SSTableWriter.java(463)
org.apache.cassandra.io.sstable.SSTableWriter: wrote index entry:
org.apache.cassandra.db.RowIndexEntry@3bb0ff0 at 106628
2013-03-28 13:44:58.071+0100 [Thread-23] [TRACE] SSTableWriter.java(145)
org.apache.cassandra.io.sstable.SSTableWriter: wrote
DecoratedKey(-4827623571007838156,
6162376162333238393739643930336266616566393039376131366238386166) at 1218191
2013-03-28 13:44:58.071+0100 [Thread-23] [TRACE] SSTableWriter.java(463)
org.apache.cassandra.io.sstable.SSTableWriter: wrote index entry:
org.apache.cassandra.db.RowIndexEntry@6e135779 at 106674
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-2-Data.db
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-2-Filter.db
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-2-TOC.txt
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting
ks-cf-tmp-ib-2-CompressionInfo.db
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] FileUtils.java(110)
org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-2-Index.db
2013-03-28 13:44:58.091+0100 [Thread-23] [DEBUG] SSTable.java(154)
org.apache.cassandra.io.sstable.SSTable: Deleted
/path/to/cassandra/data/ks/cf/ks-cf-tmp-ib-2
2013-03-28 13:44:58.091+0100 [Thread-23] [INFO] StreamInSession.java(136)
org.apache.cassandra.streaming.StreamInSession: Streaming of file
/path/to/cassandra/data/ks/cf/ks-cf-ib-2-Data.db sections=131
progress=406399/1638227 - 24% for
org.apache.cassandra.streaming.StreamInSession@37d40164 failed: requesting a
retry.
2013-03-28 13:44:58.092+0100 [Thread-23] [DEBUG] IncomingTcpConnection.java(91)
org.apache.cassandra.net.IncomingTcpConnection: IOException reading from
socket; closing
java.io.IOException: CRC unmatched
at
org.apache.cassandra.streaming.compress.CompressedInputStream.decompress(CompressedInputStream.java:111)
at
org.apache.cassandra.streaming.compress.CompressedInputStream.read(CompressedInputStream.java:79)
at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:320)
at
org.apache.cassandra.utils.BytesReadTracker.readUnsignedShort(BytesReadTracker.java:140)
at
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
at
org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:160)
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)
{noformat}
This is common for snappy and java zip:
{noformat}
at
org.apache.cassandra.streaming.compress.CompressedInputStream.read(CompressedInputStream.java:79)
at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:320)
at
org.apache.cassandra.utils.BytesReadTracker.readUnsignedShort(BytesReadTracker.java:140)
at
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
at
org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:160)
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)
{noformat}
When the download runs in single thread, the problem disappears.
> SSL problems with inter-DC communication
> ----------------------------------------
>
> Key: CASSANDRA-5391
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5391
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 1.2.3
> Environment: $ /etc/alternatives/jre_1.6.0/bin/java -version
> java version "1.6.0_23"
> Java(TM) SE Runtime Environment (build 1.6.0_23-b05)
> Java HotSpot(TM) 64-Bit Server VM (build 19.0-b09, mixed mode)
> $ uname -a
> Linux hostname 2.6.32-358.2.1.el6.x86_64 #1 SMP Tue Mar 12 14:18:09 CDT 2013
> x86_64 x86_64 x86_64 GNU/Linux
> $ cat /etc/redhat-release
> Scientific Linux release 6.3 (Carbon)
> $ facter | grep ec2
> ...
> ec2_placement => availability_zone=us-east-1d
> ...
> $ rpm -qi cassandra
> cassandra-1.2.3-1.el6.cmp1.noarch
> (custom built rpm from cassandra tarball distribution)
> Reporter: Ondřej Černoš
> Assignee: T Jake Luciani
> Priority: Blocker
>
> I get SSL and snappy compression errors in multiple datacenter setup.
> The setup is simple: 3 nodes in AWS east, 3 nodes in Rackspace. I use
> slightly modified Ec2MultiRegionSnitch in Rackspace (I just added a regex
> able to parse the Rackspace/Openstack availability zone which happens to be
> in unusual format).
> During {{nodetool rebuild}} tests I managed to (consistently) trigger the
> following error:
> {noformat}
> 2013-03-19 12:42:16.059+0100 [Thread-13] [DEBUG]
> IncomingTcpConnection.java(79)
> org.apache.cassandra.net.IncomingTcpConnection: IOException reading from
> socket; closing
> java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> at org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:78)
> at org.xerial.snappy.SnappyNative.rawUncompress(Native Method)
> at org.xerial.snappy.Snappy.rawUncompress(Snappy.java:391)
> at
> org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:93)
> at
> org.apache.cassandra.streaming.compress.CompressedInputStream.decompress(CompressedInputStream.java:101)
> at
> org.apache.cassandra.streaming.compress.CompressedInputStream.read(CompressedInputStream.java:79)
> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:337)
> at
> org.apache.cassandra.utils.BytesReadTracker.readUnsignedShort(BytesReadTracker.java:140)
> at
> org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
> at
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
> at
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:160)
> at
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
> at
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:226)
> at
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:166)
> at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:66)
> {noformat}
> The exception is raised during DB file download. What is strange is the
> following:
> * the exception is raised only when rebuildig from AWS into Rackspace
> * the exception is raised only when all nodes are up and running in AWS (all
> 3). In other words, if I bootstrap from one or two nodes in AWS, the command
> succeeds.
> Packet-level inspection revealed malformed packets _on both ends of
> communication_ (the packet is considered malformed on the machine it
> originates on).
> Further investigation raised two more concerns:
> * We managed to get another stacktrace when testing the scenario. The
> exception was raised only once during the tests and was raised when I
> throttled the inter-datacenter bandwidth to 1Mbps.
> {noformat}
> java.lang.RuntimeException: javax.net.ssl.SSLException: bad record MAC
> at com.google.common.base.Throwables.propagate(Throwables.java:160)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
> at java.lang.Thread.run(Thread.java:662)
> Caused by: javax.net.ssl.SSLException: bad record MAC
> at com.sun.net.ssl.internal.ssl.Alerts.getSSLException(Alerts.java:190)
> at
> com.sun.net.ssl.internal.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1649)
> at
> com.sun.net.ssl.internal.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1607)
> at
> com.sun.net.ssl.internal.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:859)
> at
> com.sun.net.ssl.internal.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:755)
> at
> com.sun.net.ssl.internal.ssl.AppInputStream.read(AppInputStream.java:75)
> at
> org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:151)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> ... 1 more
> {noformat}
> This is pure SSL error with no snappy interference.
> * I managed to trigger the exception during {{nodetool repair}} tests when
> replacing dead node with a new one _on the aws side_, which means the problem
> is not restricted to the one-way scenario only.
> {noformat}
> 2013-03-27 14:06:03.033+0100 [Thread-137] [INFO] StreamInSession.java(136)
> org.apache.cassandra.streaming.StreamInSession: Streaming of file
> /path/to/cassandra/data/ks/cf/ks-cf-ib-2-Data.db sections=3 progress=0/20513
> - 0% for org.apache.cassandra.streaming.StreamInSession@14450ae7 failed:
> requesting a retry.
> 2013-03-27 14:06:03.033+0100 [Thread-138] [DEBUG] FileUtils.java(110)
> org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-98-Data.db
> 2013-03-27 14:06:03.033+0100 [Thread-138] [DEBUG] FileUtils.java(110)
> org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-98-Filter.db
> 2013-03-27 14:06:03.034+0100 [Thread-138] [DEBUG] FileUtils.java(110)
> org.apache.cassandra.io.util.FileUtils: Deleting ks-cf-tmp-ib-98-TOC.txt
> 2013-03-27 14:06:03.034+0100 [Thread-137] [DEBUG]
> IncomingTcpConnection.java(91)
> org.apache.cassandra.net.IncomingTcpConnection: IOException reading from
> socket; closing
> java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> at org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:78)
> at org.xerial.snappy.SnappyNative.rawUncompress(Native Method)
> at org.xerial.snappy.Snappy.rawUncompress(Snappy.java:391)
> at
> org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:93)
> at
> org.apache.cassandra.streaming.compress.CompressedInputStream.decompress(CompressedInputStream.java:101)
> at
> org.apache.cassandra.streaming.compress.CompressedInputStream.read(CompressedInputStream.java:79)
> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:320)
> at
> org.apache.cassandra.utils.BytesReadTracker.readUnsignedShort(BytesReadTracker.java:140)
> at
> org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
> at
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
> at
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:160)
> 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)
> {noformat}
--
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