Re: Can "data_file_directories" make use of multiple disks?

2018-04-09 Thread Paulo Motta
> cassandra.yaml states that "Directories where Cassandra should store data on > disk. Cassandra will spread data evenly across them, subject to the > granularity of the configured compaction strategy.". I feel it is not correct > anymore. Is it worth updating the doc? In fact this changed aft

Re: sstableloader: Stream failed

2016-05-23 Thread Paulo Motta
Can you telnet 10.211.55.8 7000? This is the port used for streaming communication with the destination node. If not you should check what is the configured storage_port in the destination node and set that in the cassandra.yaml of the source node so it's picked up by sstableloader. 2016-05-23 10

Re: Autobootstrap in Cassandra

2016-05-23 Thread Paulo Motta
You may also check in the system.log, loaded properties are logged on node startup. 2016-05-23 19:55 GMT-03:00 Jonathan Haddad : > > find / -name 'cassandra.yaml' -exec grep -nH auto_bootstrap {} \; > > On Mon, May 23, 2016 at 3:44 PM Rajath Subramanyam > wrote: > >> Hi Cassandra users, >> >> I

Re: Error while rebuilding a node: Stream failed

2016-05-25 Thread Paulo Motta
The stack trace from the rebuild command not show the root cause of the rebuild stream error. Can you check the system.log for ERROR logs during streaming and paste here?

Re: Error while rebuilding a node: Stream failed

2016-05-25 Thread Paulo Motta
amMessage.java:44) > ~[apache-cassandra-2.1.13.jar:2.1.13] > at > org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351) > [apache-cassandra-2.1.13.jar:2.1.13] > at > org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessa

Re: Error while rebuilding a node: Stream failed

2016-05-25 Thread Paulo Motta
r.java:331) > ~[apache-cassandra-2.1.13.jar:2.1.13] > at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] > > On Wed, May 25, 2016 at 8:49 PM, Paulo Motta > wrote: > >> This is the log of the destination/rebuilding node, you need to check >> what is the erro

Re: Error while rebuilding a node: Stream failed

2016-05-25 Thread Paulo Motta
> Workaround is to set to a larger streaming_socket_timeout_in_ms **on the source node**., the new default will be 8640ms (1 day). 2016-05-25 17:23 GMT-03:00 Paulo Motta : > Was there any other ERROR preceding this on this node (in particular the > last few lines of [STREAM-IN-/172.

Re: Error while rebuilding a node: Stream failed

2016-05-25 Thread Paulo Motta
age.java:51) > ~[apache-cassandra-2.1.13.jar:2.1.13] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:250) > ~[apache-cassandra-2.1.13.jar:2.1.13] > at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] > > On W

Re: Error while rebuilding a node: Stream failed

2016-05-26 Thread Paulo Motta
114 > StreamResultFuture.java:207 - [Stream > #74c57bc0-231a-11e6-a698-1b05ac77baf9] Stream failed > > > Streaming does not seem to be resumed again from this node. Shall I just > kill again the entire rebuild process? > > On Thu, May 26, 2016 at 12:17 AM, Paulo Motta

Re: Error while rebuilding a node: Stream failed

2016-05-27 Thread Paulo Motta
I'm afraid raising streaming_socket_timeout_in_ms won't help much in this case because the incoming connection on the source node is timing out on the network layer, and streaming_socket_timeout_in_ms controls the socket timeout in the app layer and throws SocketTimeoutException (not java.io.IOExce

Re: Streaming from 1 node only when adding a new DC

2016-06-15 Thread Paulo Motta
For rebuild, replace and -Dcassandra.consistent.rangemovement=false in general we currently pick the closest replica (as indicated by the Snitch) which has the range, what will often map to the same node due to the dynamic snitch, specially when N=RF. This is good for picking a node in the same DC

Re: StreamCoordinator.ConnectionsPerHost set to 1

2016-06-16 Thread Paulo Motta
Increasing the number of threads alone won't help, because you need to add connectionsPerHost-awareness to StreamPlan.requestRanges (otherwise only a single connection per host is created) similar to what was done to StreamPlan.transferFiles by CASSANDRA-3668, but maybe bit trickier. There's an ope

Re: High Heap Memory usage during nodetool repair in Cassandra 3.0.3

2016-06-20 Thread Paulo Motta
You could also be hitting CASSANDRA-11739, which was fixed on 3.0.7 and could potentially cause OOMs for long-running repairs. 2016-06-20 13:26 GMT-03:00 Robert Stupp : > One possibility might be CASSANDRA-11206 (Support large partitions on the > 3.0 sstable format), which reduces heap usage for

Re: Exception in logs using LCS .

2016-06-28 Thread Paulo Motta
1. Not necessarily data corruption, but it seems compaction is trying to write data in the wrong order most likely due to a temporary race condition/bug a la #9935, but since the compaction fails your original data is probably safe (you can try running scrub to verify/fix corruptions). 2. This is p

Re: use private ip for internode and public IP for seeds

2016-07-27 Thread Paulo Motta
Were you able to troubleshoot this yet? Private IPs for listen_address, public IP for broadcast_address, and prefer_local=true on cassandra-rackdc.properties should be sufficient to make nodes in the same DC communicate over private address, so something must be going on there. Can you check in yo

Re: Node after restart sees other nodes down for 10 minutes

2016-07-27 Thread Paulo Motta
This looks somewhat related to CASSANDRA-9630. What is the C* version? Can you check with netstats if other nodes keep connections with the stopped node in the CLOSE_WAIT state? And also if the problem disappears if you run nodetool disablegossip before stopping the node? 2016-07-26 16:54 GMT-03:

Re: Node after restart sees other nodes down for 10 minutes

2016-07-27 Thread Paulo Motta
8df87657-c39b-405a-ba54-d60b577c1429 1d > -- > -- > ip-10-4-43-65 > Datacenter: us-east > === > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- Address Load Toke

Re: Sync failed between in AntiEntropySessions - Repair

2016-08-05 Thread Paulo Motta
It seems you have a streaming error, look for ERROR statement in the streaming classes before that which may give you a more specific root cause. In any case, I'd suggest you to upgrade to 2.1.15 as there were a couple of streaming fixes on this version that might help. 2016-08-05 11:15 GMT-03:00

Re: Sync failed between in AntiEntropySessions - Repair

2016-08-05 Thread Paulo Motta
[na:1.8.0_60] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_60] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60] > > I al looking through the changes files to see if it is a bug fixed in the > 2.1.15

Re: Sync failed between in AntiEntropySessions - Repair

2016-08-05 Thread Paulo Motta
(ConnectionHandler.java:257) > ~[apache-cassandra-2.1.14.jar:2.1.14] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60] > > We have this error with at least 5 more nodes in the same log. But the > error doesn't say that much > > > Saludos > > Jean Carlo > > "

Re: Adding Materialized View triggers "Mutation Too Large" error.

2016-08-08 Thread Paulo Motta
What happens is that when trying to rebuild the MV, the rebuilder tries to create a very large batch that exceeds commitlog_segment_size_in_mb. This limitation is currently being addressed on CASSANDRA-11670. Two options I can see to workaround this for now: 1) increase commitlog_segment_size_in_mb

Re: Incremental repairs leading to unrepaired data

2016-08-09 Thread Paulo Motta
Anticompaction throttling can be done by setting the usual compaction_throughput_mb_per_sec knob on cassandra.yaml or via nodetool setcompactionthroughput. Did you try lowering that and checking if that improves the dropped mutations? 2016-08-09 13:32 GMT-03:00 Stefano Ortolani : > Hi all, > > I

Re: nodetool repair with -pr and -dc

2016-08-10 Thread Paulo Motta
On 2.0 repair -pr option is not supported together with -local, -hosts or -dc, since it assumes you need to repair all nodes in all DCs and it will throw and error if you try to run with nodetool, so perhaps there's something wrong with range_repair options parsing. On 2.1 it was added support to

Re: Incremental repairs leading to unrepaired data

2016-08-10 Thread Paulo Motta
thing else wrong perhaps you need to increase your capacity. What version are you in? 2016-08-10 8:21 GMT-03:00 Stefano Ortolani : > Not yet. Right now I have it set at 16. > Would halving it more or less double the repair time? > > On Tue, Aug 9, 2016 at 7:58 PM, Paulo Motta > w

Re: migrating from 2.1.2 to 3.0.8 log errors

2016-08-10 Thread Paulo Motta
Another thing to note is that according to NEWS.txt upgrade from 2.1.x is only supported from version 2.1.9, so if this is not an effect of that I'm actually surprised upgrade from 2.1.2 worked without any issues. 2016-08-10 15:48 GMT-03:00 Tyler Hobbs : > That just means that a client/driver dis

Re: nodetool repair with -pr and -dc

2016-08-11 Thread Paulo Motta
-pr option ( which i suppose we should to > prevent duplicate checks) in 2.0 then if we run the repair on all nodes in > a single DC then it should be sufficient and we should not need to run it > on all nodes across DC's ? > > > > On Wed, Aug 10, 2016 at 5:01 PM, Paulo M

Re: New node block in autobootstrap

2016-08-15 Thread Paulo Motta
What version are you in? This seems like a typical case were there was a problem with streaming (hanging, etc), do you have access to the logs? Maybe look for streaming errors? Typically streaming errors are related to timeouts, so you should review your cassandra streaming_socket_timeout_in_ms and

Re: full and incremental repair consistency

2016-08-19 Thread Paulo Motta
Running repair with -local flag does not mark sstables as repaired, since you can't guarantee data in other DCs are repaired. In order to support incremental repair, you need to run a full repair without the -local flag, and then in the next time you run repair, previously repaired sstables are ski

Re: full and incremental repair consistency

2016-08-19 Thread Paulo Motta
ables or not flagged ? > > By the way, I suppose the repair flag don't break sstable file > immutability, so I wonder how it is stored. > > -- > Jérôme Mainaud > jer...@mainaud.com > > 2016-08-19 15:02 GMT+02:00 Paulo Motta : > >> Running repair with -local fl

Re: Preferred IP is NULL

2016-08-21 Thread Paulo Motta
See CASSANDRA-9748, I think it might be related. 2016-08-20 15:20 GMT-03:00 Anuj Wadehra : > Hi, > > We use multiple interfaces in multi DC setup.Broadcast address is public > IP while listen address is private IP. > > I dont understand why prefeerred IP in peers table is null for all rows. > > T

Re: Preferred IP is NULL

2016-08-22 Thread Paulo Motta
on a node? My traffic would still fail. > > I want that at least nodes in my local DC should contact at each other on > private IP. I thought preferred IP is for that purpose so focussing on > fixing the null value of preferred IPs. > > > Thanks > Anuj > > > On Sun, 21 A

Re: How to start using incremental repairs?

2016-08-25 Thread Paulo Motta
1. Migration procedure is no longer necessary after CASSANDRA-8004, and since you never ran repair before this would not make any difference anyway, so just run repair and by default (CASSANDRA-7250) this will already be incremental. 2. Incremental repair is not supported with -pr, -local or -st/-e

Re: How to start using incremental repairs?

2016-08-26 Thread Paulo Motta
't know incremental repairs were not compatible with -pr > What is the underlying reason? > > Regards, > Stefano > > > On Fri, Aug 26, 2016 at 1:25 AM, Paulo Motta > wrote: > >> 1. Migration procedure is no longer necessary after CASSANDRA-8004, and >> since

Re: How to start using incremental repairs?

2016-08-26 Thread Paulo Motta
clarifying! > > > On Fri, Aug 26, 2016 at 2:14 PM, Paulo Motta > wrote: > >> > What is the underlying reason? >> >> Basically to minimize the amount of anti-compaction needed, since with >> RF=3 you'd need to perform anti-compaction 3 times in a par

Re: nodetool repair uses option '-local' and '-pr' togather

2016-09-01 Thread Paulo Motta
https://issues.apache.org/jira/browse/CASSANDRA-7450 2016-09-01 13:11 GMT-03:00 Li, Guangxing : > Hi, > > I have a cluster running 2.0.9 with 2 data centers. I noticed that > 'nodetool repair -pr keyspace cf' runs very slow (OpsCenter shows that the > node's data size is 39 GB and the largest SST

Re: nodetool repair uses option '-local' and '-pr' togather

2016-09-02 Thread Paulo Motta
com/questions/82414/do-you-have-to-run-nodetool-repair-on-every-node. > > Thanks again. > > George > > On Thu, Sep 1, 2016 at 10:22 AM, Paulo Motta > wrote: > >> https://issues.apache.org/jira/browse/CASSANDRA-7450 >> >> 2016-09-01 13:11 GMT-03:00 Li, Guangxing :

Re: CASSANDRA-12278

2016-09-02 Thread Paulo Motta
Forwarding to the user@cassandra.apache.org list as this list is specific for cassandra-development, not general cassandra questions. Can you check the repository you built the snapshot from contains the commit 01d5fa8acf05973074482eda497677c161a311ac? Is java 1.8.0_101 on your $env:PATH ? Can yo

Re: nodetool repair uses option '-local' and '-pr' togather

2016-09-05 Thread Paulo Motta
3 > 127.0.0.5 > > > ccm node1 nodetool getendpoints replication_test sample bif > > 127.0.0.3 > 127.0.0.5 > 127.0.0.1 > > > ccm node1 nodetool getendpoints replication_test sample biz > > 127.0.0.2 > 127.0.0.3 > 127.0.0.5 > > On Fri, Sep 2, 2016 at 9

Re: Upgrade cassandra 2.1.14 to 3.0.7

2016-09-12 Thread Paulo Motta
Migration procedure is no longer required for incremental repair as of 2.1.4 since CASSANDRA-8004, which was the reason why the migration procedure was required for LCS before. The migration procedure is only useful now to skip validation on already repaired sstables in the first incremental repair

Re: How to start using incremental repairs?

2016-09-12 Thread Paulo Motta
>> An extract of this conversation should definitely be posted somewhere. >> Read a lot but never learnt all these bits... >> >> On Fri, Aug 26, 2016 at 2:53 PM, Paulo Motta >> wrote: >> >>> > I must admit that I fail to understand currently how runn

Re: Incremental repairs in 3.0

2016-09-12 Thread Paulo Motta
> I truncate a table lcs, Then I inserted one line and I used nodetool flush to have all the sstables. Using a RF 3 I ran a repair -inc directly and I observed that the value of Reaired At was equal 0. Were you able to troubleshoot this? The value of repairedAt should be mutated even when there is

Re: [ANNOUNCEMENT] Website update

2016-09-12 Thread Paulo Motta
> Are there equivalent JIRAs for the TODOs somewhere? Not that I know of, but I think you can create a github pull request for punctual doc updates and AFAIK a jira ticket will be automatically created from it. Alternatively, feel free to open a JIRA meta-ticket with subtasks for doc TODOs and pu

Re: Multiple Network Interfaces in non-EC2

2016-09-12 Thread Paulo Motta
This seems like a bug, it seems we always bind the outgoing socket to the private/listen address. Would you mind opening a JIRA and posting the link here? Thanks 2016-09-12 3:35 GMT-03:00 Amir Dafny-Man : > Hi, > > > > I followed the docs (http://docs.datastax.com/en/cassandra/3.x/cassandra/ > c

Re: How to start using incremental repairs?

2016-09-12 Thread Paulo Motta
on by just marking SSTables as being repaired (which is fast), > but the rest of the nodes will still have to perform anticompaction as they > won't share all of its token ranges. Right ? > > Cheers, > > Alex > > Le lun. 12 sept. 2016 à 13:56, Paulo Motta a > écrit :

Re: Upgrade cassandra 2.1.14 to 3.0.7

2016-09-19 Thread Paulo Motta
s why '-full' > option was added. You did the hardest part in noticing this change and > start wondering about it. > > C*heers, > --- > Alain Rodriguez - @arodream - al...@thelastpickle.com > France > > The Last Pickle - Apache Cassandra Consulting > http://w

Re: https://issues.apache.org/jira/browse/CASSANDRA-10961 fix

2016-09-20 Thread Paulo Motta
Hello Zhiyan, Replying to the mailing list since this could help others. I'm not sure what that could be, it's generally related to some kind of corruption, perhaps CASSANDRA-10791. Although the message is similar to #10971, that is restricted to streaming so it's a different issue here. Was this

Re: Repairing without -pr shows unexpected out-of-sync ranges

2016-09-22 Thread Paulo Motta
There are a couple of things that could be happening here: - There will be time differences between when nodes participating repair flush, so in write-heavy tables there will always be minor differences during validation, and those could be accentuated by low resolution merkle trees, which will aff

Re: New node block in autobootstrap

2016-09-27 Thread Paulo Motta
: > @Paulo Motta > > Even we are facing Streaming timeout exceptions during 'nodetool rebuild' > , I set streaming_socket_timeout_in_ms to 8640 (24 hours) as suggested > in datastax blog - https://support.datastax.com/h > c/en-us/articles/206502913-FAQ-How-to-reduc

Re: New node block in autobootstrap

2016-09-27 Thread Paulo Motta
> *at > org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:413)* > *at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:245)* > *at java.lang.Thread.run(Thread.java:745)* > > On Sep 27, 2016 11:

Re: Repairs at scale in Cassandra 2.1.13

2016-09-28 Thread Paulo Motta
There were a few streaming bugs fixed between 2.1.13 and 2.1.15 (see CHANGES.txt for more details), so I'd recommend you to upgrade to 2.1.15 in order to avoid having those. 2016-09-28 9:08 GMT-03:00 Alain RODRIGUEZ : > Hi Anubhav, > > >> I’m considering doing subrange repairs (https://github.com

Re: Repairing without -pr shows unexpected out-of-sync ranges

2016-10-04 Thread Paulo Motta
performing the anticompaction phase? > > Cheers, > Stefano > > On Tue, Sep 27, 2016 at 4:09 PM, Stefano Ortolani > wrote: > > Didn't know about (2), and I actually have a time drift between the > nodes. > > Thanks a lot Paulo! > > > > Regards, > >

Re: dtests jolokia fails to attach

2016-10-06 Thread Paulo Motta
I had this problem before but can't remember the root cause, but I think it was related to conflicting JVMs on the same machine. Can you check if you have more than one JVM installed and try to define JAVA_HOME if it's not defined? Maybe this is related: https://github.com/rhuss/jolokia/issues/189

Re: Slow performance after upgrading from 2.0.9 to 2.1.11

2016-10-28 Thread Paulo Motta
Haven't seen this before, but perhaps it's related to CASSANDRA-10433? This is just a wild guess as it's in a related codepath, but maybe worth trying out the patch available to see if it helps anything... 2016-10-28 15:03 GMT-02:00 Dikang Gu : > We are seeing huge cpu regression when upgrading o

Re: Storing videos in cassandra

2016-11-14 Thread Paulo Motta
For the record, there is an interesting use case of globo.com using Cassandra to store video payload and stream live video at scale (in particular, the FIFA World Cup + Olympics), but it's a pretty non-conventional/advanced use case: - https://leandromoreira.com.br/2015/04/26/fifa-2014-world-cup-li

Re: Is it a must to run Cassandra repair in scheduled time

2016-11-18 Thread Paulo Motta
This is an informative piece on (anti-entropy) repairs: https://cassandra-zone.com/understanding-repairs/ 2016-11-18 8:12 GMT-02:00 wxn...@zjqunshuo.com : > Thanks Ben for the response. It's very helpfull and it's really what I > want. > > > *From:* Ben Dalling > *Date:* 2016-11-18 18:07 > *To:*

Re: Node replacement failed in 2.2

2016-11-18 Thread Paulo Motta
What does nodetool gossipinfo shows for endpoint /2401:db00:2130:4091:face:0:13:0 ? Does it contain the TOKENS attribute? If it's missing, is it only missing on this node or other nodes as well? 2016-11-18 17:02 GMT-02:00 Dikang Gu : > Hi, I encountered couple times that I could not replace a dow

Re: Node replacement failed in 2.2

2016-11-20 Thread Paulo Motta
130:4091:face:0:13:0 shows > "TOKENS: not present", on all live nodes. It means tokens are missing, > right? What would cause this? > > Thanks. > Dikang. > > On Fri, Nov 18, 2016 at 11:15 AM, Paulo Motta > wrote: > >> What does nodetool gossipinfo shows for

Re: Is it *safe* to issue multiple replace-node at the same time?

2016-11-22 Thread Paulo Motta
It's safe but since the replacement node will stream data from a single replica per local range, it will potentially propagate any inconsistencies from the replica it streams from, so it's recommended to run repair after a replace to reduce entropy specially when replacing a node with the same IP d

Re: Extremely large ValidationExecutor.MaxPoolSize in Cassandra 2.1.13

2016-11-24 Thread Paulo Motta
This is not a problem per se, it's just the maximum number of concurrent threads allowed in the validation pool which is Integer.MAX_VALUE, which will limit the maximum number of simultaneous validations the node will handle. It may be too big, but you probably will never reach anywhere close to th

Re: Bootstrap fails on 3.10

2016-11-25 Thread Paulo Motta
If you have an MV table It seems you're hitting https://issues.apache.org/jira/browse/CASSANDRA-12905. I will bump it's priority to critical since it can prevent or difficult bootstrap. Did you try resuming bootstrap with "nodetool bootstrap resume" after the failure? It may eventually succeed, si

Re: Time range for metrics histogram

2016-12-17 Thread Paulo Motta
See CASSANDRA-11752 for 2.2+ histogram. 2016-12-17 21:13 GMT-02:00 Aleksandr Ivanov : > Hi C* experts! > > I'm trying to understand over what time range C* latency metrics histogram > is calculated. > Several sources state that max is calculated from C* start, but on graphs > I see that max laten

Re: Streaming error during repair

2017-01-05 Thread Paulo Motta
Fixed on https://issues.apache.org/jira/browse/CASSANDRA-12905 (still to be released). If you're running repair of all keyspace or tables in a single command, run each table separately and should improve things a bit. 2017-01-05 7:54 GMT-02:00 Robert Sicoie : > Hi guys, > > While running repairs,

Re: incremental repairs with -pr flag?

2017-01-11 Thread Paulo Motta
The objective of non-incremental primary-range repair is to avoid redoing work, but with incremental repair anticompaction will segregate repaired data so no extra work is done on the next repair. You should run nodetool repair [ks] [table] in all nodes sequentially. The more often you run, the sm

Re: questions about snapshots origin and removal

2015-07-17 Thread Paulo Motta
Hello John, Below are the situations where a snapshot is automatically taken on 1.2: 1. during compactions, if snapshot_before_compaction yaml option is set to true, in this case, the snapshot name will be -compact-. 2. when you drop or truncate a CF, when auto_snapshot yaml option is set to true

Re: auto_bootstrap=false broken?

2015-08-04 Thread Paulo Motta
Hello Christian, You may use the start-up parameter -Dcassandra.join_ring=false if you don't want the node to join the ring on startup. More about this parameter here: http://docs.datastax.com/en/cassandra/2.0/cassandra/tools/toolsCUtility_t.html You can later join the ring via nodetool join comm

Re: Replace dead node in non-vnode 1.2 cluster

2015-09-17 Thread Paulo Motta
The datastax documentation was fixed after the initial confusion with vnodes vs non-vnodes, so you should be safe to follow the procedure described there. Make sure to set the -Dcassandra.replace_address=address_of_dead_node JVM option (don't worry about the initial token). 2015-09-17 21:21 GMT-03

Re: Does Cassandra 2.2.1 works with Java 7?

2015-09-28 Thread Paulo Motta
Yes, the target version for 2.2 is 1.7. 2015-09-28 2:23 GMT-04:00 Lu, Boying : > Hi, All, > > > > The latest stable release of Cassandra is 2.2.1 and I notice the following > line in the “Requirements” section of README.asc comes with the source > codes: > > Java >=1.7 (OpenJDK and Oracle JVMS h

Re: broadcast address on EC2 without Elastic IPs.

2015-09-30 Thread Paulo Motta
You may get a better/faster response by asking directly on the java driver list: https://groups.google.com/a/lists.datastax.com/forum/#!forum/java-driver-user 2015-09-30 10:24 GMT-04:00 Renato Perini : > Hello! > I have configured a small cluster composed of three nodes on Amazon EC2. > The 3 mac

Re: Why can't nodetool status include a hostname?

2015-10-08 Thread Paulo Motta
Have you tried using the -r or --resolve-ip option? 2015-10-07 19:59 GMT-07:00 Kevin Burton : > I find it really frustrating that nodetool status doesn't include a > hostname > > Makes it harder to track down problems. > > I realize it PRIMARILY uses the IP but perhaps cassandra.yml can include >

Re: [cassandra 2.1.3] Missing host ID

2015-10-08 Thread Paulo Motta
Hello Eduardo, Your node is trying to write a hint to another node (after a timed out write), but because of some race condition it does not have its token table updated soon after startup, so it cannot locate the node with that ID. You should not be worried, as the only consequence is that one hi

Re: unchecked_tombstone_compaction - query

2015-10-15 Thread Paulo Motta
Hello Deepak, The dev@cassandra list is exclusive for development announcements and discussions, so I will reply to users@cassandra as someone else might have a similar question. Basically, there is pre-check, that defines which sstables are eligible for single-sstable tombstone compaction, and a

Re: Repair Hangs while requesting Merkle Trees

2015-11-23 Thread Paulo Motta
The issue might be related to the ESTABLISHED connections just in one end. I don't think it might be related to inter_dc_tcp_nodelay or request_timeout_in_ms options. Did you restart the process when you changed the request_timeout_in_ms option? This might be why the problem got fixed and not the o

Re: Upgrade instructions don't make sense

2015-11-23 Thread Paulo Motta
This actually means if you *DON'T* use vnodes, you should be careful to set num_tokens in the cassandra.yaml when upgrading, since the default cassandra.yaml is to have vnodes enabled. You should ignore these instructions if you already have a vnodes-enabled cluster, and set num_tokens accordingly.

Re: Unable to use using multiple network interfaces in Cassandra 3.0

2015-11-25 Thread Paulo Motta
Hello Sergey, Currently Cassandra listens in one interface (the listen_address), so you can only use multiple interfaces if your NAT configuration can route from your public IP address to your private interface, as typically happens on EC2 and other clouds. We're currently working to support liste

Re: Change the rack of a server

2015-11-26 Thread Paulo Motta
Changing the rack of a live node is discouraged, since the ring ranges the node is responsible for will change, meaning the node will not own part of the data for its new ranges and other nodes may not have some of its current data. It will be a forbidden operation in the upcoming versions of Cas

Re: Questions to StorageServiceMBean.forceRepaireRangeAsync()

2015-11-30 Thread Paulo Motta
Hello Boying, 1. "repairedAt" actually means "fullRepair", so set that to true if you want to run ordinary/full repair or false if you want to run incremental repair. 2. You should use StorageServiceMBean.repairAsync(String, Map), where the options map will be parsed by org.apache.cassandra.repair

Re: Issues on upgrading from 2.2.3 to 3.0

2015-12-03 Thread Paulo Motta
You can migrate from the RackInferingSnitch to PropertiesFileSnitch by populating the cassandra-topology.properties with the same rack/dc assignments of the previous snitch (what you can't change is the assignment, but you can change snitches if you maintain the same assignments as before). For ex

Re: [RELEASE] Apache Cassandra 3.1 released

2015-12-10 Thread Paulo Motta
> Will 3.2 contain the bugfixes that are in 3.0.2 as well? If the bugfix affects both 3.2 and 3.0.2, yes. Otherwise it will only go in the affected version. > Is 3.x.y just 3.0.x plus new stuff? Where most of the time y is 0, unless there's a really serious issue that needs fixing? You can't rea

Re: Query Consistency Issues...

2015-12-15 Thread Paulo Motta
What cassandra and driver versions are you running? It may be that the second update is getting the same timestamp as the first, or even a lower timestamp if it's being processed by another server with unsynced clock, so that update may be getting lost. If you have high frequency updates in the s

Re: Query Consistency Issues...

2015-12-15 Thread Paulo Motta
ill have to be sync'ed for this to work, right? correct, you may also use ntp to synchronize clocks between clients. 2015-12-15 12:19 GMT-08:00 James Carman : > > > On Tue, Dec 15, 2015 at 2:57 PM Paulo Motta > wrote: > >> What cassandra and driver versions are you running

Re: What are the best ways to learn Apache Cassandra

2015-12-19 Thread Paulo Motta
After you're acquainted with the basic stuff a good set of intermediate and advanced resources are the presentations from the last Cassandra summits with real world tips and applications. http://pt.slideshare.net/planetcassandra/clipboards/cassandra-summit-2015 https://www.youtube.com/playlist?lis

Re: Data rebalancing algorithm

2015-12-24 Thread Paulo Motta
The new node will own some parts (ranges) of the ring according to the ring tokens the node is responsible for. These tokens are defined from the yaml property initial_token (manual assignment) or num_tokens (random assignment). During the bootstrap process raw data from sstables sections containi

Re: ClosedChannelExcption while nodetool repair

2016-01-12 Thread Paulo Motta
You may be running into https://issues.apache.org/jira/browse/CASSANDRA-10961, which will be fixed in 2.2.5. In the meantime, you may replace your cassandra jar with a snapshot version available in that issue. 2016-01-12 10:38 GMT-03:00 Jan Kesten : > Hi, > > I have some problems recently on my c

Re: broadcast_address in multi data center setups

2016-01-18 Thread Paulo Motta
broadcast_address is the address exposed for internal inter-node communication, while rpc_address is the address that will listen to clients. all nodes need to talk to each other via the broadcast_address, so if they are within the same network, you may use public or private IPs as broadcast_addre

Re: Run Repairs when a Node is Down

2016-01-18 Thread Paulo Motta
Hello Anuj, Repairing a range with down replicas may be valid if there is still QUORUM up replicas and using at least QUORUM for writes. My understanding is that it was disabled as default behavior on CASSANDRA-2290 to avoid misuse/confusion, and its not advisable to remain with a down node for a

Re: Logging

2016-01-25 Thread Paulo Motta
What kind of actions? nodetool/system actions or cql statements? You could probably achieve identity-based logging with logback Mapped Diagnostic Context (MDC - logback.qos.ch/manual/mdc.html), but you'd need to patch your own Cassandra jars in many locations to provide that information to the log

Re: Logging

2016-01-25 Thread Paulo Motta
ext-users-in-logback-and-spring-security/ > ? > Would that work? Or you would rather suggest to go MDC way? > > Thanks, > > Oleg > > On Mon, Jan 25, 2016 at 9:23 AM, Paulo Motta > wrote: > >> What kind of actions? nodetool/system actions or cql statements?

Re: 3k sstables during a repair incremental !!

2016-02-10 Thread Paulo Motta
Are you using vnodes by any chance? If so, how many? How many nodes and what's the replication factor? How was data inserted (at what consistency level)? Streaming might create a large number of sstables with vnodes (see CASSANDRA-10495), so in case data is inconsistent between nodes (detected dur

Re: Debugging write timeouts on Cassandra 2.2.5

2016-02-10 Thread Paulo Motta
Are you using the same GC settings as the staging 2.0 cluster? If not, could you try using the default GC settings (CMS) and see if that changes anything? This is just a wild guess, but there were reports before of G1-caused instabilities with small heap sizes (< 16GB - see CASSANDRA-10403 for more

Re: Faster version of 'nodetool status'

2016-02-12 Thread Paulo Motta
There was a recent performance inefficiency in nodetool status with virtual nodes that will be fixed in the next releases (CASSANDRA-7238), so it should be faster with this fixed. You can also query StorageServiceMBean.getLiveNodes() via JMX (jolokia or some other jmx client). For a list of useful

Re: how to read parent_repair_history table?

2016-02-25 Thread Paulo Motta
Hello Jimmy, The parent_repair_history table keeps track of start and finish information of a repair session. The other table repair_history keeps track of repair status as it progresses. So, you must first query the parent_repair_history table to check if a repair started and finish, as well as

Re: how to read parent_repair_history table?

2016-02-25 Thread Paulo Motta
d, it will have multiple entries, one for each keysapce_name value. > no ? > > thanks > > > > Sent from my iPhone > > On Feb 25, 2016, at 5:48 AM, Paulo Motta wrote: > > Hello Jimmy, > > The parent_repair_history table keeps track of start and finish > infor

Re: how to read parent_repair_history table?

2016-02-25 Thread Paulo Motta
lag i can tell the difference? > or does it actualy matter? > > thanks > > > > > Sent from my iPhone > > On Feb 25, 2016, at 10:37 AM, Paulo Motta > wrote: > > > why each job repair execution will have 2 entries? I thought it will be > one entry

Re: How to complete bootstrap with exception due to stream failure?

2016-02-28 Thread Paulo Motta
Were the columns sensor.lastEvaluation and sensordb.lastCheckTime dropped by any chance? If so, you might be hitting https://issues.apache.org/jira/browse/CASSANDRA-11050, fixed in upcoming 3.4. If that's the case, you may want to check if nodes other than 192.168.10.10 have the dropped columns in

Re: how to read parent_repair_history table?

2016-03-03 Thread Paulo Motta
ed and not the node being used to > comparing the data, correct? > > > On Thu, Feb 25, 2016 at 1:38 PM, Paulo Motta > wrote: > >> > how does it work when repair job targeting only local vs all DC? is >> there any columns or flag i can tell the difference? or does

Re: How to measure the write amplification of C*?

2016-03-10 Thread Paulo Motta
This is a good source on Cassandra + write amplification: http://www.slideshare.net/rbranson/cassandra-and-solid-state-drives 2016-03-10 9:57 GMT-03:00 Benjamin Lerer : > Cassandra should not cause any write amplification. Write amplification > appends only when you updates data on SSDs. Cassandr

Re: Cassandra causing OOM Killer to strike on new cluster running 3.4

2016-03-13 Thread Paulo Motta
You could be hitting CASSANDRA-11344 ( https://issues.apache.org/jira/browse/CASSANDRA-11344). If that's the case, you may try to replace your cassandra jar on an affected node with a version with this fix in place and force bloom filter regeneration to see if if it fixes your problem. You can bui

Re: Cassandra 3.2.1: Memory leak?

2016-03-14 Thread Paulo Motta
Can you check with nodetool tpstats if bloom filter mem space utilization is very large/ramping up before the node gets killed? You could be hitting CASSANDRA-11344. 2016-03-12 19:43 GMT-03:00 Mohamed Lrhazi : > In my case, all nodes seem to be constantly logging messages like these: > > DEBUG [G

Re: Cassandra 3.2.1: Memory leak?

2016-03-14 Thread Paulo Motta
> HINT 0 > MUTATION 0 > COUNTER_MUTATION 0 > BATCH_STORE 0 > BATCH_REMOVE 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > [root@avesterr

Re: C* memory leak during compaction

2016-03-15 Thread Paulo Motta
Did you check bloom filter sizes with nodetool tablestats to see if you're hitting CASSANDRA-11344? If that`s the case there's a patch available along with instructions in some other recent thread on how to fix it. 2016-03-15 6:49 GMT-03:00 ssiv...@gmail.com : > Duplicate the answer from Russell

  1   2   >