Thanks Eric. Happy new year 2015 for all Cassandra developers and Users :). This group seems the most active of apache big data projects.
Will come back with more questions :) Thanks Ajay On Dec 31, 2014 8:02 PM, "Eric Stevens" <migh...@gmail.com> wrote: > You can totally avoid the impact of tombstones by rotating your partition > key in the exact counts table, and only deleting whole partitions once > you've counted them. Once you've counted them you never have cause to read > that partition key again. > > You can totally store the final counts in Cassandra as a standard > (non-counter) column, and you can even use counters to keep track of the > time slices which haven't been formally counted yet so that you can get > reasonably accurate information about time slices that haven't been trued > up yet. > > This is basically what's called a Lambda architecture - use efficient real > time processing to get pretty close to accurate values when real time > performance matters, then use a cleanup process to get perfectly accurate > values when you can afford non-real-time processing times, and store that > final computation so that you can continue to access it quickly. > > > is there any technical reason behind it (just out of curiosity)? > > Distributed counting is a fundamentally hard problem if you wish to do so > in a manner that avoids bottlenecks (i.e. not distributed) and also > provides for perfect accuracy. There's plenty of research in this area, > and there isn't a single algorithm that provides for all the properties we > would hope for. Instead there are different algorithms that make different > tradeoffs. > > The way that Cassandra's counters can fail is that most operations in > Cassandra are idempotent - if we're not sure whether an update has been > applied correctly or not, we can simply apply it again, because it's safe > to do twice. Counters are not idempotent. If you try to increment a > counter, and you're not certain whether the increment was successful or > not, it is *not* safe to try again (if it was successful the previous > time, you've now incremented twice when it should have been once). > > Most of the time counters are reasonable and accurate, but in failure > scenarios you may get some changes applied more than once, or not at all. > With that in mind, you might find that being perfectly accurate most of the > time, and being within a fraction of a percent the other times is > acceptable. If so, counters are your friend, and if not, a more complex > lambda style approach as we've been advocating here is best. > > On Tue, Dec 30, 2014 at 10:54 PM, Ajay <ajay.ga...@gmail.com> wrote: > >> Thanks Janne and Rob. >> >> The idea is like this : To store the User clicks on Cassandra and a >> scheduler to count/aggregate the clicks per link or ad >> hourly/daily/monthly and store in My SQL (or may be in Cassandra itself). >> Since tombstones will be deleted only after some days (as per >> configuration), could the subsequent queries to count the rows get affected >> (I mean say thousands of tombstones will affect the performance of the >> query) ? >> >> Secondly as I understand from this mail thread, the counter is not >> correct for this use case, is there any technical reason behind it (just >> out of curiosity)? >> >> Thanks >> Ajay >> >> On Tue, Dec 30, 2014 at 10:37 PM, Janne Jalkanen < >> janne.jalka...@ecyrd.com> wrote: >> >>> >>> Hi! >>> >>> Yes, since all the writes for a partition (or row if you speak Thrift) >>> always go to the same replicas, you will need to design to avoid hotspots - >>> a pure day row will cause all the writes for a single day to go to the same >>> replicas, so those nodes will have to work really hard for a day, and then >>> the next day it’s again hard work for some other nodes. If you have an >>> user id there in front, then it would distribute better. >>> >>> For tombstone purposes think of your access patterns; if you have a >>> date-based system, it probably does not matter since you will scan those >>> UUIDs once, and then they will be tombstoned away. It’s cleaner if you can >>> delete the entire row with a single command, but as long as you never read >>> it again, I don’t think this matters much. >>> >>> The real problems with wide rows come with compaction, and you shouldn’t >>> have much problems with compaction because this is an append-only row, so >>> it should be fine as a fairly wide row. Make some back-of-the-envelope >>> calculations and if it looks like you’re going to be hitting tens of >>> millions of columns per day, then store per hour. >>> >>> One important thing: in order not to lose clicks, always use timeuuids >>> instead of timestamps (or else two clicks coming in for the same id would >>> overwrite itself and count as one). >>> >>> /Janne >>> >>> On 30 Dec 2014, at 06:28, Ajay <ajay.ga...@gmail.com> wrote: >>> >>> Thanks Janne, Alain and Eric. >>> >>> Now say I go with counters (hourly, daily, monthly) and also store UUID >>> as below: >>> >>> user Id : yyyy/mm/dd as row key and dynamic columns for each click with >>> column key as timestamp and value as empty. Periodically count the columns >>> and rows and correct the counters. Now in this case, there will be one row >>> per day but as many columns as user click. >>> >>> Other way is to store row per hour >>> user id : yyyy/mm/dd/hh as row key and dynamic columns for each click >>> with column key as timestamp and value as empty. >>> >>> Is there any difference (in performance or any known issues) between >>> more rows Vs more columns as Cassandra deletes them through tombstones (say >>> by default 20 days). >>> >>> Thanks >>> Ajay >>> >>> On Mon, Dec 29, 2014 at 7:47 PM, Eric Stevens <migh...@gmail.com> wrote: >>> >>>> > If the counters get incorrect, it could't be corrected >>>> >>>> You'd have to store something that allowed you to correct it. For >>>> example, the TimeUUID approach to keep true counts, which are slow to read >>>> but accurate, and a background process that trues up your counter columns >>>> periodically. >>>> >>>> On Mon, Dec 29, 2014 at 7:05 AM, Ajay <ajay.ga...@gmail.com> wrote: >>>> >>>>> Thanks for the clarification. >>>>> >>>>> In my case, Cassandra is the only storage. If the counters get >>>>> incorrect, it could't be corrected. For that if we store raw data, we can >>>>> as well go that approach. But the granularity has to be as seconds level >>>>> as >>>>> more than one user can click the same link. So the data will be huge with >>>>> more writes and more rows to count for reads right? >>>>> >>>>> Thanks >>>>> Ajay >>>>> >>>>> >>>>> On Mon, Dec 29, 2014 at 7:10 PM, Alain RODRIGUEZ <arodr...@gmail.com> >>>>> wrote: >>>>> >>>>>> Hi Ajay, >>>>>> >>>>>> Here is a good explanation you might want to read. >>>>>> >>>>>> >>>>>> http://www.datastax.com/dev/blog/whats-new-in-cassandra-2-1-a-better-implementation-of-counters >>>>>> >>>>>> Though we use counters for 3 years now, we used them from start C* >>>>>> 0.8 and we are happy with them. Limits I can see in both ways are: >>>>>> >>>>>> Counters: >>>>>> >>>>>> - accuracy indeed (Tend to be small in our use case < 5% - when the >>>>>> business allow 10%, so fair enough for us) + we recount them through a >>>>>> batch processing tool (spark / hadoop - Kind of lambda architecture). So >>>>>> our real-time stats are inaccurate and after a few minutes or hours we >>>>>> have >>>>>> the real value. >>>>>> - Read-Before-Write model, which is an anti-pattern. Makes you use >>>>>> more machine due to the pressure involved, affordable for us too. >>>>>> >>>>>> Raw data (counted) >>>>>> >>>>>> - Space used (can become quite impressive very fast, depending on >>>>>> your business) ! >>>>>> - Time to answer a request (we expose the data to customer, they >>>>>> don't want to wait 10 sec for Cassandra to read 1 000 000 + columns) >>>>>> - Performances in o(n) (linear) instead of o(1) (constant). Customer >>>>>> won't always understand that for you it is harder to read 1 than 1 000 >>>>>> 000, >>>>>> since it should be reading 1 number in both case, and your interface will >>>>>> have very unstable read time. >>>>>> >>>>>> Pick the best solution (or combination) for your use case. Those >>>>>> disadvantages lists are not exhaustive, just things that came to my mind >>>>>> right now. >>>>>> >>>>>> C*heers >>>>>> >>>>>> Alain >>>>>> >>>>>> 2014-12-29 13:33 GMT+01:00 Ajay <ajay.ga...@gmail.com>: >>>>>> >>>>>>> Hi, >>>>>>> >>>>>>> So you mean to say counters are not accurate? (It is highly likely >>>>>>> that multiple parallel threads trying to increment the counter as users >>>>>>> click the links). >>>>>>> >>>>>>> Thanks >>>>>>> Ajay >>>>>>> >>>>>>> >>>>>>> On Mon, Dec 29, 2014 at 4:49 PM, Janne Jalkanen < >>>>>>> janne.jalka...@ecyrd.com> wrote: >>>>>>> >>>>>>>> >>>>>>>> Hi! >>>>>>>> >>>>>>>> It’s really a tradeoff between accurate and fast and your read >>>>>>>> access patterns; if you need it to be fairly fast, use counters by all >>>>>>>> means, but accept the fact that they will (especially in older >>>>>>>> versions of >>>>>>>> cassandra or adverse network conditions) drift off from the true click >>>>>>>> count. If you need accurate, use a timeuuid and count the rows (this >>>>>>>> is >>>>>>>> fairly safe for replays too). However, if using timeuuids your storage >>>>>>>> will need lots of space; and your reads will be slow if the click >>>>>>>> counts >>>>>>>> are huge (because Cassandra will need to read every item). Using >>>>>>>> counters >>>>>>>> makes it easy to just grab a slice of the time series data and shove >>>>>>>> it to >>>>>>>> a client for visualization. >>>>>>>> >>>>>>>> You could of course do a hybrid system; use timeuuids and then >>>>>>>> periodically count and add the result to a regular column, and then >>>>>>>> remove >>>>>>>> the columns. Note that you might want to optimize this so that you >>>>>>>> don’t >>>>>>>> end up with a lot of tombstones, e.g. by bucketing the writes so that >>>>>>>> you >>>>>>>> can delete everything with just a single partition delete. >>>>>>>> >>>>>>>> At Thinglink some of the more important counters that we use are >>>>>>>> backed up by the actual data. So for speed purposes we use always >>>>>>>> counters >>>>>>>> for reads, but there’s a repair process that fixes the counter value >>>>>>>> if we >>>>>>>> suspect it starts drifting off the real data too much. (You might be >>>>>>>> able >>>>>>>> to tell that we’ve been using counters for quite some time :-P) >>>>>>>> >>>>>>>> /Janne >>>>>>>> >>>>>>>> On 29 Dec 2014, at 13:00, Ajay <ajay.ga...@gmail.com> wrote: >>>>>>>> >>>>>>>> > Hi, >>>>>>>> > >>>>>>>> > Is it better to use Counter to User click count than maintaining >>>>>>>> creating new row as user id : timestamp and count it. >>>>>>>> > >>>>>>>> > Basically we want to track the user clicks and use the same for >>>>>>>> hourly/daily/monthly report. >>>>>>>> > >>>>>>>> > Thanks >>>>>>>> > Ajay >>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >>> >> >