[jira] [Comment Edited] (CASSANDRA-14616) cassandra-stress write hangs with default options
[ https://issues.apache.org/jira/browse/CASSANDRA-14616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16578028#comment-16578028 ] Jeremy edited comment on CASSANDRA-14616 at 8/13/18 10:07 AM: -- Hello, I would like to try solving this issue. I have done some preliminary testing and it appears that it is caused by cassandra-stress waiting for uncertainty to stabilize, the trace from jstack is included below. {quote} java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00076d873e20> (a java.util.concurrent.CountDownLatch$Sync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231) at org.apache.cassandra.stress.util.Uncertainty$WaitForTargetUncertainty.await(Uncertainty.java:56) at org.apache.cassandra.stress.util.Uncertainty.await(Uncertainty.java:85) at org.apache.cassandra.stress.report.StressMetrics.waitUntilConverges(StressMetrics.java:135) at org.apache.cassandra.stress.StressAction.run(StressAction.java:269) at org.apache.cassandra.stress.StressAction.warmup(StressAction.java:121) at org.apache.cassandra.stress.StressAction.run(StressAction.java:70) at org.apache.cassandra.stress.Stress.run(Stress.java:143) at org.apache.cassandra.stress.Stress.main(Stress.java:62) {quote} I also did some printlns for debugging in 3.11. {quote} uncertainty: NaN targetUncertainty: 0.02 measurements: 1 minMeasurements: 30 measurements: 1 maxMeasurements: 200 uncertainty: NaN targetUncertainty: 0.02 measurements: 2 minMeasurements: 30 measurements: 2 maxMeasurements: 200 ... uncertainty: NaN targetUncertainty: 0.02 measurements: 200 minMeasurements: 30 measurements: 200 maxMeasurements: 200 {quote} In the warmup phase, the program aims for either uncertainty to fall below 0.02 with at least 30 measurements or to hit 200 measurements. It ends up waiting for 200 measurements since the uncertainty is always NaN. The same problem doesn't occur in 3.0 because the Runnable (https://github.com/apache/cassandra/blob/cassandra-3.0/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java#L86) calls wakeAll after 2 iterations. However, uncertainty is still always NaN in 3.0. The problem arises in 3.11 and trunk as that runnable loop was refactored into reportingLoop which waited for all 200 tries first. https://github.com/apache/cassandra/blob/cassandra-3.11/tools/stress/src/org/apache/cassandra/stress/report/StressMetrics.java#L154 Here's what it looks like for 3.0. {quote} Warming up WRITE with 0 iterations... ___ Updated value: NaN uncertainty: NaN targetUncertainty: 0.02 measurements: 1 minMeasurements: 30 measurements: 1 maxMeasurements: 200 ___ Updated value: NaN uncertainty: NaN targetUncertainty: 0.02 measurements: 2 minMeasurements: 30 measurements: 2 maxMeasurements: 200 latch counted down via wakeall wakeAll via line 123 in stressmetrics WARNING: uncertainty mode (err<) results in uneven workload between thread runs, so should be used for high level analysis only Running with 4 threadCount {quote} I think this is being caused by having 0 iterations for warmup. The number of iterations is decided at the start by {{Math.min(5, (int) (settings.command.count * 0.25)) * settings.node.nodes.size();}}. https://github.com/apache/cassandra/blob/cassandra-3.11/tools/stress/src/org/apache/cassandra/stress/StressAction.java#L108 . When {{./tools/bin/cassandra-stress write}} is called without any arguments, settings.command.count evaluates to -1 and {{Math.min(5, (int) (settings.command.count * 0.25)) * settings.node.nodes.size();}} evaluates to 0 so we always end up with 0 iterations. One proposed fix is to choose a minimum nonzerovalue for iterations in the warmup phase. Something like https://github.com/yarnspinnered/cassandra/commit/33cf059f63b56ac17a3f66869615d3d7cc52f8a9 . I tried this and it no longer hangs but I'm not sure on the exact value or if there is a better way to fix this. was (Author: yarnspinner): Hello, I would like to try solving this issue. I have done some preliminary testing and it appears that it is caused by cassandra-stress waiting for uncertainty to stabilize, the trace from jstack is included below. {quote} java.lang.Thread.State: WAITING (parking) at
[jira] [Commented] (CASSANDRA-14616) cassandra-stress write hangs with default options
[ https://issues.apache.org/jira/browse/CASSANDRA-14616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16578028#comment-16578028 ] Jeremy commented on CASSANDRA-14616: Hello, I would like to try solving this issue. I have done some preliminary testing and it appears that it is caused by cassandra-stress waiting for uncertainty to stabilize, the trace from jstack is included below. {quote} java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00076d873e20> (a java.util.concurrent.CountDownLatch$Sync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231) at org.apache.cassandra.stress.util.Uncertainty$WaitForTargetUncertainty.await(Uncertainty.java:56) at org.apache.cassandra.stress.util.Uncertainty.await(Uncertainty.java:85) at org.apache.cassandra.stress.report.StressMetrics.waitUntilConverges(StressMetrics.java:135) at org.apache.cassandra.stress.StressAction.run(StressAction.java:269) at org.apache.cassandra.stress.StressAction.warmup(StressAction.java:121) at org.apache.cassandra.stress.StressAction.run(StressAction.java:70) at org.apache.cassandra.stress.Stress.run(Stress.java:143) at org.apache.cassandra.stress.Stress.main(Stress.java:62) {quote} I also did some printlns for debugging in 3.11. {quote} uncertainty: NaN targetUncertainty: 0.02 measurements: 1 minMeasurements: 30 measurements: 1 maxMeasurements: 200 uncertainty: NaN targetUncertainty: 0.02 measurements: 2 minMeasurements: 30 measurements: 2 maxMeasurements: 200 ... uncertainty: NaN targetUncertainty: 0.02 measurements: 200 minMeasurements: 30 measurements: 200 maxMeasurements: 200 {quote} In the warmup phase, the program aims for either uncertainty to fall below 0.02 or to hit 200 measurements. It ends up waiting for 200 measurements since the uncertainty is always NaN. The same problem doesn't occur in 3.0 because the Runnable (https://github.com/apache/cassandra/blob/cassandra-3.0/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java#L86) calls wakeAll after 2 iterations. However, uncertainty is still always NaN in 3.0. The problem arises in 3.11 and trunk as that runnable loop was refactored into reportingLoop which waited for all 200 tries first. https://github.com/apache/cassandra/blob/cassandra-3.11/tools/stress/src/org/apache/cassandra/stress/report/StressMetrics.java#L154 Here's what it looks like for 3.0. {quote} Warming up WRITE with 0 iterations... ___ Updated value: NaN uncertainty: NaN targetUncertainty: 0.02 measurements: 1 minMeasurements: 30 measurements: 1 maxMeasurements: 200 ___ Updated value: NaN uncertainty: NaN targetUncertainty: 0.02 measurements: 2 minMeasurements: 30 measurements: 2 maxMeasurements: 200 latch counted down via wakeall wakeAll via line 123 in stressmetrics WARNING: uncertainty mode (err<) results in uneven workload between thread runs, so should be used for high level analysis only Running with 4 threadCount {quote} I think this is being caused by having 0 iterations for warmup. The number of iterations is decided at the start by {{Math.min(5, (int) (settings.command.count * 0.25)) * settings.node.nodes.size();}}. https://github.com/apache/cassandra/blob/cassandra-3.11/tools/stress/src/org/apache/cassandra/stress/StressAction.java#L108 . When {{./tools/bin/cassandra-stress write}} is called without any arguments, settings.command.count evaluates to -1 and {{Math.min(5, (int) (settings.command.count * 0.25)) * settings.node.nodes.size();}} evaluates to 0 so we always end up with 0 iterations. One proposed fix is to choose a minimum nonzerovalue for iterations in the warmup phase. Something like https://github.com/yarnspinnered/cassandra/commit/33cf059f63b56ac17a3f66869615d3d7cc52f8a9 . I tried this and it no longer hangs but I'm not sure on the exact value or if there is a better way to fix this. > cassandra-stress write hangs with default options > - > > Key: CASSANDRA-14616 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14616 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Priority: Major > > Cassandra stress sits there for
[jira] [Commented] (CASSANDRA-14522) sstableloader options assume the rpc/native interface is the same as the internode interface
[ https://issues.apache.org/jira/browse/CASSANDRA-14522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16564827#comment-16564827 ] Jeremy commented on CASSANDRA-14522: Hello, I reproduced the error on my computer by setting different ip addresses for rpc_address, listen_address and local_address on my local cassandra instance. I then fixed it for 3.0 by querying system.peers and system.local to map the rpc_address to the listen_address in NativeSSTableLoaderClient.init. Is this a good way to go about it? Diff: https://github.com/apache/cassandra/compare/trunk...yarnspinnered:14522-3.0 The tests pass except a few which failed due to timeout. However, these tests are unrelated and also fail on a clean copy cloned from the repo. Tests like 'testUpdateColumnNotInViewWithFlush, testClusteringKeyMultiColumnRestrictions, testClusteringOrder'. > 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) >
[jira] [Commented] (CASSANDRA-14522) sstableloader options assume the rpc/native interface is the same as the internode interface
[ https://issues.apache.org/jira/browse/CASSANDRA-14522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539392#comment-16539392 ] Jeremy commented on CASSANDRA-14522: [~aweisberg] Thank you very much for the explanation! I will try out the testing setup and see how it goes. > 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 >
[jira] [Commented] (CASSANDRA-14522) sstableloader options assume the rpc/native interface is the same as the internode interface
[ https://issues.apache.org/jira/browse/CASSANDRA-14522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 endpoints = metadata.getReplicas(Metadata.quote(keyspace), tokenRange); Range 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 >
[jira] [Commented] (CASSANDRA-14522) sstableloader options assume the rpc/native interface is the same as the internode interface
[ https://issues.apache.org/jira/browse/CASSANDRA-14522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16529969#comment-16529969 ] Jeremy commented on CASSANDRA-14522: Hello I would like to try working on this issue > 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 >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 >