incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Dr. Martin Grabmüller <Martin.Grabmuel...@eleven.de>
Subject RE: skip + limit support in GetSlice
Date Mon, 06 Sep 2010 08:34:38 GMT
Have you considered creating a second column family which acts as an index for
the original column family?  Have the record number as the column name, and the
value as the identifier (primary key) of the original data, and do a 
 
1.  get_slice(<index_column_family>, start='00051235', finish='', limit=100)
2.  get_slice(<original_column_family>, columns=<list of returned column values>)
 
This way, only 100 columns are returned on the first call, and 100 columns (or super columns)
on the second.  You have two calls instead of one, but it should be faster because
much less data is transferred (and the latency can be hidden by concurrency).
 
Martin


________________________________

	From: Michal Augustýn [mailto:augustyn.michal@gmail.com] 
	Sent: Monday, September 06, 2010 10:26 AM
	To: user@cassandra.apache.org
	Subject: Re: skip + limit support in GetSlice
	
	
	Hi Mike, 

	yes, I read the PDF to the finish. Twice. As I wrote, my application is not accessed by users,
it's accessed by other applications that can access pages randomly.

	So when some application wants to get page 51235 (so skip is 5123500, limit is 100) then
I have to:

	1) GetSlice(from: "", to: "", limit: 5123500)
	2) Read only the last column name.
	3) GetSlice(from: point2value, to: "", limit: 100)

	The problem is in 1) - Cassandra has to read 5123500 columns, serialize them, send them using
 Thrift protocol and deserialize them. Finally, I throw 5123499 of columns away. It doesn't
seem to be very efficient.

	So I'm looking for another solution for this scenario. I know the right way for pagination
in Cassandra and I'm using them if I can...

	So if this kind of pagination cannot be added to standard Cassandra Thrift API then I should
create some separate Thrift API that will handle my scenario (and avoid high network traffic).
Am I right?

	Thanks!

	Augi


	2010/9/5 Mike Peters <cassandra@softwareprojects.com>
	

		Hi Michal,
		
		Did you read the PDF Stu sent over, start to finish?  There are several different approaches
described there.
		
		With Cassandra, what we found works best for pagination:
		* Keep a separate 'total_records' count and increment/decrement it on every insert/delete
		* When getting slices, pass 'last seen' as the 'from' and keep the 'to' empty.  Pass the
number of records you want to show per page in the 'count'.
		* Avoid letting user skip to page X, using Next/Prev/First/Last only (same way GMail does
it) 



		Michal Augustýn wrote: 

			I know that "Prev/Next" is good solution for web applications. But when I want to access
data from another application or when I want to access pages randomly... 

			I don't know the internal structure of memtables etc., so I don't know if columns in row
are indexable. If now, then I just want to transfer my workaround to server (to avoid huge
network traffic)...
			
			
			2010/9/5 Stu Hood <stu.hood@rackspace.com>
			

				Cassandra supports the recommended approach from: http://www.percona.com/ppc2009/PPC2009_mysql_pagination.pdf
				
				For large numbers of items, skip + limit is extremely inefficent.
				

				-----Original Message-----
				From: "Michal Augustýn" <augustyn.michal@gmail.com>
				Sent: Sunday, September 5, 2010 5:39am
				To: user@cassandra.apache.org
				Subject: skip + limit support in GetSlice
				
				Hello,
				
				probably this is feature request. Simply, I would like to have support for
				standard pagination (skip + limit) in GetSlice Thrift method. Is this
				feature on the road map?
				
				Now, I have to perform GetSlice call, that starts on "" and "limit" is set
				to "skip" value. Then I read the last column name returned and subsequently
				perform the final GetSlice call - I use the last column name as "start" and
				set "limit" to "limit" value.
				
				This workaround is not very efficient when I need to skip a lot of columns
				(so "skip" is high) - then a lot of data must be transferred via network. So
				I think that support for Skip in GetSlice would be very useful (to avoid
				high network traffic).
				
				The implementation could be very straightforward (same as the workaround) or
				maybe it could be more efficient - I think that whole row (so all columns)
				must fit into memory so if we have all columns in memory...
				
				Thank you!
				
				Augi
				
				
				





Mime
View raw message