cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ariel Weisberg (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-14522) sstableloader options assume the rpc/native interface is the same as the internode interface
Date Mon, 30 Jul 2018 17:23:00 GMT

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

Ariel Weisberg edited comment on CASSANDRA-14522 at 7/30/18 5:22 PM:
---------------------------------------------------------------------

There are (almost) no silly questions.

You can run a cluster of Cassandra nodes on a single machine using https://github.com/riptano/ccm


We write automated tests that use ccm called dtests https://github.com/apache/cassandra-dtest
which I don't think you will have to work with for this.

You will need some familiarity with Python and virtual environments and setting up requirements
files. We use Python 3 now for the tests but cqlsh is Python 2. Let me know if you need help.

For testing this you don't need multiple Cassandra processes, but you would need to setup
up a second loopback interface that isn't reachable from your usual one so if it picks the
incorrect outbound interface it fails to connect. You can probably do this with a firewall
rule.

So that code you are looking at isn't the problem code I think. It's picking the correct interface
to connect to based on the cluster metadata from the driver, but it's probably picking the
wrong interface to connect on in BulkLoadConnectionFactory. On trunk this code has changed
so I am not sure if it's still an issue https://github.com/apache/cassandra/commit/fc92db2b9b56c143516026ba29cecdec37e286bb#diff-fc44570de4b634df61bd83b639db98d4L51
and where things go wrong is buried a little deeper. https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/streaming/StreamSession.java#L197

[~jasobrown] ^ I think this may be an issue with Netty  intracluster streaming and security?
Is it going to pick the wrong address just like the bulk loader? I am assuming not because
the server loads the YAML and it all works out fine even in an AWS like setup.

In 3.0 where this is reported it is https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java#L51
and it's calling FBUtilities.getLocalAddress() to pick the outbound interface. 

Looking at the ticket I see connection refused so it's reaching the other node, but the other
node isn't bound. Just be aware if you test this you will get a different result if there
is no route.

TL;DR in all cases I think what we want is FBUtilities.getLocalAddress() to be used to provide
a default interface, but there is a command line option to specify the outbound interface
for streaming. In 3.0 and other versions forwards (3.11) and backwards (2.2, 2.1) this propagates
into BulkLoadConnectionFactory. In trunk/4.0 it will have to propagate all the way into where
StreamSession is calling FBUtilities.getLocalAddress(). One option might be to manually set
the value in FBUtilities. You will need to create and test a patch for each version.


was (Author: aweisberg):
There are (almost) no silly questions.

You can run a cluster of Cassandra nodes on a single machine using https://github.com/riptano/ccm


We write automated tests that use ccm called dtests https://github.com/apache/cassandra-dtest
which I don't think you'll have to work with for this.

You will need some familiarity with Python and virtual environments and setting up requirements
files. We use Python 3 now for the tests but cqlsh is Python 2. Let me know if you need help.

For testing this you don't need multiple Cassandra processes, but you would need to setup
up an a second loopback interface that isn't reachable from your usual one so if it picks
the incorrect outbound interface it fails to connect. You can probably do this with a firewall
rule.

So that code you are looking at isn't the problem code I think. It's picking the correct interface
to connect to based on the cluster metadata from the driver, but it's probably picking the
wrong interface to connect on in BulkLoadConnectionFactory. On trunk this code has changed
so I am not sure if it's still an issue https://github.com/apache/cassandra/commit/fc92db2b9b56c143516026ba29cecdec37e286bb#diff-fc44570de4b634df61bd83b639db98d4L51
and where things go wrong is buried a little deeper. https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/streaming/StreamSession.java#L197

[~jasobrown] ^ I think this may be an issue with Netty  intracluster streaming and security?
Is it going to pick the wrong address just like the bulk loader? I am assuming not because
the server loads the YAML and it all works out fine even in an AWS like setup.

In 3.0 where this is reported it is https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java#L51
and it's calling FBUtilities.getLocalAddress() to pick the outbound interface. 

Looking at the ticket I see connection refused so it's reaching the other node, but the other
node isn't bound. Just be aware if you test this you will get a different result if there
is no route.

TL;DR in all cases I think what we want is FBUtilities.getLocalAddress() to be used to provide
a default interface, but there is a command line option to specify the outbound interface
for streaming. In 3.0 and other versions forwards (3.11) and backwards (2.2, 2.1) this propagates
into BulkLoadConnectionFactory. In trunk/4.0 it will have to propagate all the way into where
StreamSession is calling FBUtilities.getLocalAddress(). One option might be to manually set
the value in FBUtilities. You will need to create and test a patch for each version.

> sstableloader options assume the rpc/native interface is the same as the internode interface
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-14522
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14522
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Tools
>            Reporter: Jeremy Hanna
>            Assignee: Jeremy
>            Priority: Major
>              Labels: lhf
>
> Currently, in the LoaderOptions for the BulkLoader, the user can give a list of initial
host addresses.  That's to do the initial connection to the cluster but also to stream the
sstables.  If you have two physical interfaces, one for rpc, the other for internode traffic,
then bulk loader won't currently work.  It will throw an error such as:
> {quote}
> > sstableloader -v -u cassadmin -pw xxx -d 10.133.210.101,10.133.210.102,10.133.210.103,10.133.210.104
/var/lib/cassandra/commitlog/backup_tmp/test_bkup/bkup_tbl
> Established connection to initial hosts
> Opening sstables and calculating sections to stream
> Streaming relevant part of /var/lib/cassandra/commitlog/backup_tmp/test_bkup/bkup_tbl/mc-1-big-Data.db
/var/lib/cassandra/commitlog/backup_tmp/test_bkup/bkup_tbl/mc-2-big-Data.db  to [/10.133.210.101,
/10.133.210.103, /10.133.210.102, /10.133.210.104]
> progress: total: 100% 0  MB/s(avg: 0 MB/s)ERROR 10:16:05,311 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60]
Streaming error occurred
> java.net.ConnectException: Connection refused
>         at sun.nio.ch.Net.connect0(Native Method) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:454) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:446) ~[na:1.8.0_101]
>         at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:648) ~[na:1.8.0_101]
>         at java.nio.channels.SocketChannel.open(SocketChannel.java:189) ~[na:1.8.0_101]
>         at org.apache.cassandra.tools.BulkLoadConnectionFactory.createConnection(BulkLoadConnectionFactory.java:60)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.createConnection(StreamSession.java:266)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.ConnectionHandler.initiate(ConnectionHandler.java:86)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.start(StreamSession.java:253)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamCoordinator$StreamSessionConnector.run(StreamCoordinator.java:212)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_101]
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
~[netty-all-4.0.54.Final.jar:4.0.54.Final]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> ERROR 10:16:05,312 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60] Streaming error occurred
> java.net.ConnectException: Connection refused
>         at sun.nio.ch.Net.connect0(Native Method) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:454) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:446) ~[na:1.8.0_101]
>         at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:648) ~[na:1.8.0_101]
>         at java.nio.channels.SocketChannel.open(SocketChannel.java:189) ~[na:1.8.0_101]
>         at org.apache.cassandra.tools.BulkLoadConnectionFactory.createConnection(BulkLoadConnectionFactory.java:60)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.createConnection(StreamSession.java:266)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.ConnectionHandler.initiate(ConnectionHandler.java:86)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.start(StreamSession.java:253)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamCoordinator$StreamSessionConnector.run(StreamCoordinator.java:212)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_101]
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
~[netty-all-4.0.54.Final.jar:4.0.54.Final]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> ERROR 10:16:05,312 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60] Streaming error occurred
> java.net.ConnectException: Connection refused
>         at sun.nio.ch.Net.connect0(Native Method) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:454) ~[na:1.8.0_101]
>         at sun.nio.ch.Net.connect(Net.java:446) ~[na:1.8.0_101]
>         at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:648) ~[na:1.8.0_101]
>         at java.nio.channels.SocketChannel.open(SocketChannel.java:189) ~[na:1.8.0_101]
>         at org.apache.cassandra.tools.BulkLoadConnectionFactory.createConnection(BulkLoadConnectionFactory.java:60)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.createConnection(StreamSession.java:266)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.ConnectionHandler.initiate(ConnectionHandler.java:86)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamSession.start(StreamSession.java:253)
~[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at org.apache.cassandra.streaming.StreamCoordinator$StreamSessionConnector.run(StreamCoordinator.java:212)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_101]
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
[cassandra-all-3.0.15.2128.jar:3.0.15.2128]
>         at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
~[netty-all-4.0.54.Final.jar:4.0.54.Final]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> progress: total: 100% 0  MB/s(avg: 0 MB/s)WARN  10:16:05,320 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60]
Stream failed
> Streaming to the following hosts failed:
> [/10.133.210.101, /10.133.210.103, /10.133.210.102]
> java.util.concurrent.ExecutionException: org.apache.cassandra.streaming.StreamException:
Stream failed
>         at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:299)
>         at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:286)
>         at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
>         at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:122)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at com.datastax.bdp.tools.ShellToolWrapper.main(ShellToolWrapper.java:34)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
>         at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
>         at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310)
>         at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457)
>         at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
>         at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
>         at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
>         at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:215)
>         at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
>         at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:449)
>         at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:549)
>         at org.apache.cassandra.streaming.StreamSession.start(StreamSession.java:259)
>         at org.apache.cassandra.streaming.StreamCoordinator$StreamSessionConnector.run(StreamCoordinator.java:212)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>         at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:745)
> WARN  10:16:05,322 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60] Stream failed
> WARN  10:16:05,322 [Stream #9ed00130-6ff6-11e8-965c-93a78bf96e60] Stream failed
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message