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

Jeremy commented on CASSANDRA-14522:
------------------------------------

Sorry for any silly questions. This is my first time working on a big open 
source project and I have an idea or two about how to fix it but I'm not sure 
if it's completely off-track. Would really appreciate osme help.

I poked around in the code and it seems that the sstableloader starts out by 
launching org.apache.cassandra.tools.BulkLoader which then creates an 
ExternalClient which is a NativeSSTableLoaderClient extended with 2 fields. The 
only place i see the storagePort used in there is 

{code:java}
try (Cluster cluster = builder.build(); Session session = cluster.connect()){
for (TokenRange tokenRange : tokenRanges)
            {
                Set<Host> endpoints = 
metadata.getReplicas(Metadata.quote(keyspace), tokenRange);
                Range<Token> range = new 
Range<>(tokenFactory.fromString(tokenRange.getStart().getValue().toString()),
                                                 
tokenFactory.fromString(tokenRange.getEnd().getValue().toString()));
                for (Host endpoint : endpoints)
                {
                    int portToUse;
                    if (allowServerPortDiscovery)
                    {
                        portToUse = 
endpoint.getBroadcastAddressOptPort().portOrElse(storagePort);
                    }
                    else
                    {
                        portToUse = storagePort;
                    }
                    addRangeForEndpoint(range, 
InetAddressAndPort.getByNameOverrideDefaults(endpoint.getAddress().getHostAddress(),
 portToUse));
                }
            }
}
{code}

So I think there might be 2 ways to fix it?

#
Instead of using endpoint.getAddress().getHostAddress(), use 
endpoint.getBroadcastAddressOptPort().address?

#
Use the session to execute a CQLSH query on the system.peers table and then 
parse the broadcast addresses from there?

Also, is there a way to test this on a single node? Or would I need to go get 2 
AWS nodes and configure broadcast addresses that differ from the RPC address 
before testing it on them?

Thank you!

> 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-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to