Re: Compaction and total disk space used for highly overwritten CF
You will only have tombstones in your data if you issue deletes. What you are seeing is an artifact of the fundamental way Cassandra stores data. Once data is written to disk it is never modified. If you overwrite a column value that has already been committed to disk the old value is not changed. Instead the new value is held in memory and some time later it is written to a new file (more info here http://thelastpickle.com/2011/04/28/Forces-of-Write-and-Read/) Compaction not only kersplats data that has been deleted, it kapows data that has been over written. (See this link for a dramatic first person re-creation of compaction removing an overwritten value http://goo.gl/4TrB6 ) By overwriting all the data so often you are somewhat fighting against the server But there are some things you can try (am assuming 0.8.6, some general background http://www.datastax.com/docs/0.8/operations/tuning) * reduce the min_compaction_threshold on the CF so that data on disk gets compacted more frequently. * look at the logs to too see why / when memtables are been flushed, look for lines like INFO [ScheduledTasks:1] 2011-10-02 22:32:20,092 ColumnFamilyStore.java (line 1128) Enqueuing flush of Memtable-NoCache_Ascending@921142878(2175000/13267958 serialized/live bytes, 43500 ops) or WARN [ScheduledTasks:1] 2011-10-02 22:32:20,084 GCInspector.java (line 143) Heap is 0.778906484049155 full. You may need to reduce memtable and/or cache sizes. Cassandra will now flush up to the two largest memtables to free up memory. Adjust flush_largest_memtables_at threshold in cassandra.yaml if you don't want Cassandra to do this automatically * The memtable will be flushed to disk for 1 of 3 reasons: * The Heap is too full and cassandra wants to free memory * It has passed the memtable_operations CF threshold for changes, increase this value to flush less * It has passed the memtable_throughput CF threshold for throughput, increase this value to flush less (background http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/) * is possible reduce the amount of overwrites. Hope that helps. - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 6/10/2011, at 2:42 PM, Derek Andree wrote: We have a very hot CF which we use essentially as a durable memory cache for our application. It is about 70MBytes in size after being fully populated. We completely overwrite this entire CF every few minutes (not delete). Our hope was that the CF would stay around 70MB in size, but it grows to multiple Gigabytes in size rather quickly (less than an hour). I've heard that doing major compactions using nodetool is no longer recommended, but when we force a compaction on this CF using nodetool compact, then perform GC, size on disk shrinks to the expected 70MB. I'm wondering if we are doing something wrong here, we thought we were avoiding tombstones since we are just overwriting each column using the same keys. Is the fact that we have to do a GC to get the size on disk to shrink significantly a smoking gun that we have a bunch of tombstones? We've row cached the entire CF to make reads really fast, and writes are definitely fast enough, it's this growing disk space that has us concerned. Here's the output from nodetool cfstats for the CF in question (hrm, I just noticed that we still have a key cache for this CF which is rather dumb): Column Family: Test SSTable count: 4 Space used (live): 309767193 Space used (total): 926926841 Number of Keys (estimate): 275456 Memtable Columns Count: 37510 Memtable Data Size: 15020598 Memtable Switch Count: 22 Read Count: 4827496 Read Latency: 0.010 ms. Write Count: 1615946 Write Latency: 0.095 ms. Pending Tasks: 0 Key cache capacity: 15 Key cache size: 55762 Key cache hit rate: 0.030557854052177317 Row cache capacity: 15 Row cache size: 68752 Row cache hit rate: 1.0 Compacted row minimum size: 925 Compacted row maximum size: 1109 Compacted row mean size: 1109 Any insight appreciated. Thanks, -Derek
Hinted handoff question
I'am using Cassandra 0.7.7 and have a question about hinted handoff. I have a cluster of three nodes. I stop node 3. I see that the hint count for node 3 increases on node 1 (countPendingHints = 28709). However, when I start node 3 again, I cannot see anything in the log regarding hinted handoff delivery. If I look at the countPendingHints on node 1, it still says 28709. hinted_handoff_enabled is enabled. Thanks!
Re: Hinted handoff question
Check that node 1 sees node 3 as UP (via ring). Check the tpstats on node 1, is there an active HH task ? Take another crawl through the logs. Cheers - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 6/10/2011, at 10:35 PM, Rene Kochen wrote: I'am using Cassandra 0.7.7 and have a question about hinted handoff. I have a cluster of three nodes. I stop node 3. I see that the hint count for node 3 increases on node 1 (countPendingHints = 28709). However, when I start node 3 again, I cannot see anything in the log regarding hinted handoff delivery. If I look at the countPendingHints on node 1, it still says 28709. hinted_handoff_enabled is enabled. Thanks!
RE: Hinted handoff question
Node 3 is up (using ring on node 1). There is no HH task (active = 0, pending = 0, completed = 0, blocked = 0). This is the log from node 1 when nodes 3 starts: 2011-10-06 12:36:54,985 INFO 12:36:54,985 Node /172.16.108.19 has restarted, now UP again 2011-10-06 12:36:54,985DEBUG 12:36:54,985 Node /172.16.108.19 state normal, token 170141183460469 2011-10-06 12:36:54,985 INFO 12:36:54,985 Node /172.16.108.19 state jump to normal 2011-10-06 12:36:54,985DEBUG 12:36:54,985 clearing cached endpoints 2011-10-06 12:36:54,985DEBUG 12:36:54,985 clearing cached endpoints 2011-10-06 12:36:54,985DEBUG 12:36:54,985 No bootstrapping or leaving nodes - empty pending ranges for Traxis 2011-10-06 12:36:54,985DEBUG 12:36:54,985 attempting to connect to /172.16.108.19 2011-10-06 12:37:06,672DEBUG 12:37:06,672 Disseminating load info ... 2011-10-06 12:38:06,688DEBUG 12:38:06,688 Disseminating load info ... 2011-10-06 12:39:06,703DEBUG 12:39:06,703 Disseminating load info ... The countPendingHints on node 1 is {172.16.108.19=5176} Thanks Rene From: aaron morton [mailto:aa...@thelastpickle.com] Sent: donderdag 6 oktober 2011 12:13 To: user@cassandra.apache.org Subject: Re: Hinted handoff question Check that node 1 sees node 3 as UP (via ring). Check the tpstats on node 1, is there an active HH task ? Take another crawl through the logs. Cheers - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 6/10/2011, at 10:35 PM, Rene Kochen wrote: I'am using Cassandra 0.7.7 and have a question about hinted handoff. I have a cluster of three nodes. I stop node 3. I see that the hint count for node 3 increases on node 1 (countPendingHints = 28709). However, when I start node 3 again, I cannot see anything in the log regarding hinted handoff delivery. If I look at the countPendingHints on node 1, it still says 28709. hinted_handoff_enabled is enabled. Thanks!
Batch writes getting slow
Hi guys, We're currently testing an application against a very high load, which runs against Cassandra 0.6.13 (I know, we just never got the time to upgrade). The nature of our app is that it will write to two different SuperColumnFamilies in bursts, and to some other columnfamilies less frequently. About every 10 mins a very large number of subcolumns will be written to those superColumnFamilies ( 1 million). The writes are mainly overwrites, there are no deletes. when I do a cfstats on the SuperColumnFamilies I see something like this: SSTable count: 4 Space used (live): 1611778835 Space used (total): 1611778835 Memtable Columns Count: 82844 Memtable Data Size: 4439246 Memtable Switch Count: 5262 Read Count: 4633810 Read Latency: 3.727 ms. Write Count: 29739347 Write Latency: 0.028 ms. Pending Tasks: 0 Which tells me the Write Latency is not extremely high, but I can see in my logs that some batch writes, which contain about 1000 entries, take over 2 seconds to execute. I can see that during the times the writing gets slow there are ~3000 pending tasks, but they disappear quickly. Is there anything I can do to tune Cassandra so the writes won't take that long? Maybe increasing the MemtableOperationsInMillions so the memtables get flushed less often? Or is the load just too high and should I solve this by adding more nodes? We are currently testing with 3, replication factor 2. I can also see that Cassandra gradually takes more and more memory, eventually filling up the 16GB that is assigned to it, although it doesn't go out of memory. Is this normal behaviour? I expected to see more of a sawtooth... All help is appreciated. Thanks, Stefan
Re: Batch writes getting slow
On Thu, Oct 6, 2011 at 10:09 AM, Stefan Reek ste...@unitedgames.com wrote: I can see that during the times the writing gets slow there are ~3000 pending tasks, but they disappear quickly. Your best bet is to make the write load more constant and less bursty. If you really do need to handle bursts like that with low latency, then you probably do need more hardware. (But make sure you've covered the basics first, like putting commitlog on a separate device.) I can also see that Cassandra gradually takes more and more memory, eventually filling up the 16GB that is assigned to it, although it doesn't go out of memory. Is this normal behaviour? I expected to see more of a sawtooth... Yes, that is normal from the OS's view of the JVM. If you want to see the sawtooth you'd need to look at the JVM's internal metrics, e.g., with jconsole. -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of DataStax, the source for professional Cassandra support http://www.datastax.com
Re: Batch writes getting slow
On 10/06/2011 05:26 PM, Jonathan Ellis wrote: On Thu, Oct 6, 2011 at 10:09 AM, Stefan Reekste...@unitedgames.com wrote: I can see that during the times the writing gets slow there are ~3000 pending tasks, but they disappear quickly. Your best bet is to make the write load more constant and less bursty. If you really do need to handle bursts like that with low latency, then you probably do need more hardware. (But make sure you've covered the basics first, like putting commitlog on a separate device.) We really need to make sure that all writes are succesfully written before the next batch has to be written, so the bursts are unavoidable I think. We do have the commitlogs on separate devices, are there any other basics that I could have forgotten, or any parameters that are important for write performance? As I understand it the flush thresholds mainly influence read performance instead of write performance. Would it make any difference to write the data with more threads from the client, as that's something we can easily tune. I can also see that Cassandra gradually takes more and more memory, eventually filling up the 16GB that is assigned to it, although it doesn't go out of memory. Is this normal behaviour? I expected to see more of a sawtooth... Yes, that is normal from the OS's view of the JVM. If you want to see the sawtooth you'd need to look at the JVM's internal metrics, e.g., with jconsole. I can see the sawtooth in the JVM only for Par Eden and Par Survivor space, the CMS Old Gen space just keeps on growing though. Anyway, thanks for the quick reply. Regards, Stefan
Re: Batch writes getting slow
On Thu, Oct 6, 2011 at 10:53 AM, Stefan Reek ste...@unitedgames.com wrote: We do have the commitlogs on separate devices, are there any other basics that I could have forgotten, or any parameters that are important for write performance? 1.0 write performance is something like 30% better... I don't think there's anything else you'll find for free. As I understand it the flush thresholds mainly influence read performance instead of write performance. It can affect write performance if you're flushing really small sstables, but I doubt that's the problem here. Would it make any difference to write the data with more threads from the client, as that's something we can easily tune. Not in this case because Cassandra turns the batch into single-row writes internally, so it gets parallelized that way. If you can avoid waiting for One Big Batch and stream changes in as they happen, that would help. I can see the sawtooth in the JVM only for Par Eden and Par Survivor space, the CMS Old Gen space just keeps on growing though. Is it actually filling up enough to trigger an old-gen CMS gc? -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of DataStax, the source for professional Cassandra support http://www.datastax.com
read on multiple SS tables
Lets assume I perform frequent insert update on a column family.. Over a period of time multiple sstables will have this row/column data. I have 2 questions about how reads work in cassandra w.r.t. multiple SS tables. -If you perform a query for a specific row key and a column name, does it read the most recent SSTable first and if it finds a hit, does it stop there or does it need to read through all the SStables (to find most recent one) regardless of whether if found a hit on the most recent SSTable or not? - If I perform a slice query on a column range does cassandra iterate all the SS tables? We have an option to create 1st option: Key1 | COL1 | COL2 | COL3 . multiple columns We need to perform a slice query to get COL1-COL3 using key1. 2nd option: Key1 | COL as one column and have application place values of COL1-COLN in this one column This key would be updated several times where the app would manage adding multiple values to the one column key. Our max col value size will be less than 64mb. When you need to search for a value, we would read the one column and the application would manage looking up the appropriate value in the list of values. So I am wondering which option would be most efficient from read point of view. thanks Ramesh
CF design
I was hoping someone could share their opinions on the following CF designs or suggest a better way of doing it. My app is constantly receiving new data that contains URLs. I was thinking of hashing this URL to form a key. The data is a JSON object with several properties. For now many of its properties will be ignored and only 4 are of interests, URL, title, username, user_rating. Often times the same URL is received but shared by a different user. I’m wondering if anyone can suggest a better approach to what I propose below which will be able answer the following . Queries: I’ll be asking the following questions: 1. Give me the N most frequently shared items over : a) The last 30 minutes b) The last 24hrs c) Between D1 and D2 (where D1 and D2 represents the start and end date of interest) 2) Give me the N most shared item over the 3 time periods above WHERE the average user rating is above 5 3) Give me X for the item with the ID 123 (where X is a property for the item with the ID 123) Proposed approach Use timestamps as keys in the CF, that should take care of queries under 1 and partially handle 2 and use each column to store the JSON data, minus the common fields such as the title which will be the same no matter how many users share the same link (they’ll have their own columns in the row) other column names will be the user’s username and the value for those columns will be any JSON left over that’s not specific to the user. For the rest of 2, I can get the N items we’re interested in and calculate the average user rating for each item client side. Of course using timestamp as key means we need to maintain an index of the “real” keys/IDs to each item which would allow us to answer “Give me the item with the ID 123” Finally to address 3, I was thinking; Using the index get the timestamp of the item, and on the client side find the property of interest. CF1 Timestamp1 Title value ID ID1 Username3 {“rating”:5} Username2 {“rating”:0} Username2 {“rating”:4} Timestamp2 Title Value1 ID ID2 Username24 {“rating”:1} Username87 {“rating”:9} Username7 {“rating”:2} CF2 ID1 Timestamp1 ID2 Timestamp2 In the Username column, I'd ideally like to avoid storing the other properties as a JSON but I couldn't think of a way of doing it sensibly when that JSON grows into having 10 other properties.Does this sound like a sensible approach to designing my CFs?
cassandra.bat install
I've got the 1.0 rc2 binaries, but it looks like somebody forgot to include the Apache Daemon in the zip. According to the batch file there should be a bin\daemon directory, with a prunsrv executable in there. Cheers, Steve
Re: cassandra.bat install
It looks like it's missing from the binary distribution. If you download the source distribution it will be present. I'll see if I can get it included in the source build. Ben On Thu, Oct 6, 2011 at 2:07 PM, Stephen Pope stephen.p...@quest.com wrote: I’ve got the 1.0 rc2 binaries, but it looks like somebody forgot to include the Apache Daemon in the zip. According to the batch file there should be a bin\daemon directory, with a prunsrv executable in there. ** ** Cheers, Steve
Re: Hinted handoff question
At debug level you should see some log messages such as… Checking remote schema before delivering hints… Sleeping {}ms to stagger hint delivery… Endpoint {} died before hint delivery, aborting… Started hinted handoff for endpoint… If you want to move on, you can deliver the hints using JMX or the node should start to deliver them if you restart it (i think). Cheers - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 6/10/2011, at 11:40 PM, Rene Kochen wrote: Node 3 is up (using ring on node 1). There is no HH task (active = 0, pending = 0, completed = 0, blocked = 0). This is the log from node 1 when nodes 3 starts: 2011-10-06 12:36:54,985 INFO 12:36:54,985 Node /172.16.108.19 has restarted, now UP again 2011-10-06 12:36:54,985DEBUG 12:36:54,985 Node /172.16.108.19 state normal, token 170141183460469 2011-10-06 12:36:54,985 INFO 12:36:54,985 Node /172.16.108.19 state jump to normal 2011-10-06 12:36:54,985DEBUG 12:36:54,985 clearing cached endpoints 2011-10-06 12:36:54,985DEBUG 12:36:54,985 clearing cached endpoints 2011-10-06 12:36:54,985DEBUG 12:36:54,985 No bootstrapping or leaving nodes - empty pending ranges for Traxis 2011-10-06 12:36:54,985DEBUG 12:36:54,985 attempting to connect to /172.16.108.19 2011-10-06 12:37:06,672DEBUG 12:37:06,672 Disseminating load info ... 2011-10-06 12:38:06,688DEBUG 12:38:06,688 Disseminating load info ... 2011-10-06 12:39:06,703DEBUG 12:39:06,703 Disseminating load info ... The countPendingHints on node 1 is {172.16.108.19=5176} Thanks Rene From: aaron morton [mailto:aa...@thelastpickle.com] Sent: donderdag 6 oktober 2011 12:13 To: user@cassandra.apache.org Subject: Re: Hinted handoff question Check that node 1 sees node 3 as UP (via ring). Check the tpstats on node 1, is there an active HH task ? Take another crawl through the logs. Cheers - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 6/10/2011, at 10:35 PM, Rene Kochen wrote: I'am using Cassandra 0.7.7 and have a question about hinted handoff. I have a cluster of three nodes. I stop node 3. I see that the hint count for node 3 increases on node 1 (countPendingHints = 28709). However, when I start node 3 again, I cannot see anything in the log regarding hinted handoff delivery. If I look at the countPendingHints on node 1, it still says 28709. hinted_handoff_enabled is enabled. Thanks!
Re: read on multiple SS tables
-If you perform a query for a specific row key and a column name, does it read the most recent SSTable first and if it finds a hit, does it stop there or does it need to read through all the SStables (to find most recent one) regardless of whether if found a hit on the most recent SSTable or not? Reads all SSTables, as the only way to know which column instance has the highest time stamp is to read them all. - If I perform a slice query on a column range does cassandra iterate all the SS tables? All SSTables that contain any data for the row. (background http://thelastpickle.com/2011/04/28/Forces-of-Write-and-Read/) So I am wondering which option would be most efficient from read point of view. I would go with the first, 64MB columns will be a pain. Cheers - Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 7/10/2011, at 7:50 AM, Ramesh Natarajan wrote: Lets assume I perform frequent insert update on a column family.. Over a period of time multiple sstables will have this row/column data. I have 2 questions about how reads work in cassandra w.r.t. multiple SS tables. -If you perform a query for a specific row key and a column name, does it read the most recent SSTable first and if it finds a hit, does it stop there or does it need to read through all the SStables (to find most recent one) regardless of whether if found a hit on the most recent SSTable or not? - If I perform a slice query on a column range does cassandra iterate all the SS tables? We have an option to create 1st option: Key1 | COL1 | COL2 | COL3 . multiple columns We need to perform a slice query to get COL1-COL3 using key1. 2nd option: Key1 | COL as one column and have application place values of COL1-COLN in this one column This key would be updated several times where the app would manage adding multiple values to the one column key. Our max col value size will be less than 64mb. When you need to search for a value, we would read the one column and the application would manage looking up the appropriate value in the list of values. So I am wondering which option would be most efficient from read point of view. thanks Ramesh
Re: read on multiple SS tables
On Thu, Oct 6, 2011 at 3:56 PM, aaron morton aa...@thelastpickle.com wrote: -If you perform a query for a specific row key and a column name, does it read the most recent SSTable first and if it finds a hit, does it stop there or does it need to read through all the SStables (to find most recent one) regardless of whether if found a hit on the most recent SSTable or not? Reads all SSTables, as the only way to know which column instance has the highest time stamp is to read them all. Until https://issues.apache.org/jira/browse/CASSANDRA-2498 which makes this much faster. -Brandon
Re: Solandra distributed search
The partitioner that comes with Solandra keeps groups of documents together on the same node. The number of documents that stick together and define a shard is set by the solandra.maximum.docs.per.shard property. The shards.at.once property relates to throughput. If you are indexing to one shard at a time you will effectively be writing to just one node (since the docs are grouped together). So if you have 4 nodes and set shards.at.once set =4 you will load into all nodes at once. Once thing to consider, the more shards you have the more requests you make when you execute a search. So you don't want 100 shards with 5 nodes. On Thu, Oct 6, 2011 at 6:39 PM, Jacob, Arun arun.ja...@disney.com wrote: does the Solandra specific partitioner distribute data relatively equally across nodes? Is this influenced by the shards.at.once property? If I'm writing to 3 nodes, how would the default setting of 4 for this property affect the distribution of data across my nodes? From: Jake Luciani jak...@gmail.com Reply-To: user@cassandra.apache.org user@cassandra.apache.org Date: Mon, 15 Aug 2011 12:03:22 -0700 To: user@cassandra.apache.org user@cassandra.apache.org Subject: Re: Solandra distributed search Solandra manages the shard parameters for you. you don't need to specify anything. On Mon, Aug 15, 2011 at 3:00 PM, Jeremiah Jordan jeremiah.jor...@morningstar.com wrote: When using Solandra, do I need to use the Solr sharding synxtax in my queries? I don't think I do because Cassandra is handling the sharding, not Solr, but just want to make sure. The Solandra wiki references the distributed search limitations, which talks about the shard syntax further down the page. From what I see with how it is implemented I should just be able to pick a random Solandra node and do my query, since they are all backed by the same Cassandra data store. Correct? Thanks! -Jeremiah -- http://twitter.com/tjake -- http://twitter.com/tjake
Solandra: connection refused errors
I'm seeing this error when trying to insert data into a core I've defined in Solandra INFO [pool-7-thread-319] 2011-10-06 16:21:34,328 HttpMethodDirector.java (line 445) Retrying request INFO [pool-7-thread-1070] 2011-10-06 16:21:34,328 HttpMethodDirector.java (line 445) Retrying request INFO [pool-7-thread-335] 2011-10-06 16:21:34,327 HttpMethodDirector.java (line 439) I/O exception (java.net.ConnectException) caught when processing request: Connection refused INFO [pool-7-thread-335] 2011-10-06 16:21:34,329 HttpMethodDirector.java (line 445) Retrying request ERROR [1926426205@qtp-673795938-11] 2011-10-06 16:21:34,327 SolrException.java (line 151) org.apache.solr.common.SolrException: org.apache.solr.client.solrj.SolrServerException: java.net.ConnectException: Connection refused Has anyone seen this behavior before, the problem is that it seems to be intermittent (if it were failing all of the time, I would suspect a port or IP misconfiguration).
invalid shard name encountered
I'm seeing this in my logs: WARN [1832199239@qtp-673795938-0] 2011-10-06 16:15:46,424 CassandraIndexManager.java (line 364) invalid shard name encountered: WDPRO-NGELOG-DEV 1 WDPRO-NGELOG-DEV is the name of the index I'm creating. Is there a restriction on characters in the name?
Re: Hector has a website
very good job! 2011/10/7 Patricio Echagüe patric...@gmail.com Hi, I wanted to let you all know that Hector client has a website. http://hector-client.org There are links to documentation, Javadoc and resources from the community. If you have a personal blog and want us to include the link, let us know. Feedback is always welcome. Thanks! Hector Team. -- Jérémy