Re: strange get_range_slices behaviour v0.6.1
He there, I'm still getting odd behavior with get_range_slices. I've created a JUNIT test that illustrates the case. Could someone take a look and either let me know where my understanding is wrong or is this is a real issue? I added the following to ColumnFamilyStoreTest.java private ColumnFamilyStore insertKey1Key2Key3() throws IOException, ExecutionException, InterruptedException { List rms = new LinkedList(); RowMutation rm; rm = new RowMutation("Keyspace2", "key1".getBytes()); rm.add(new QueryPath("Standard1", null, "Column1".getBytes()), "asdf".getBytes(), 0); rms.add(rm); rm = new RowMutation("Keyspace2", "key2".getBytes()); rm.add(new QueryPath("Standard1", null, "Column1".getBytes()), "asdf".getBytes(), 0); rms.add(rm); rm = new RowMutation("Keyspace2", "key3".getBytes()); rm.add(new QueryPath("Standard1", null, "Column1".getBytes()), "asdf".getBytes(), 0); rms.add(rm); return Util.writeColumnFamily(rms); } @Test public void testThreeKeyRangeAll() throws IOException, ExecutionException, InterruptedException { ColumnFamilyStore cfs = insertKey1Key2Key3(); IPartitioner p = StorageService.getPartitioner(); RangeSliceReply result = cfs.getRangeSlice(ArrayUtils.EMPTY_BYTE_ARRAY, Util.range(p, "key1", "key3"), 10, null, Arrays.asList("Column1".getBytes())); assertEquals(3, result.rows.size()); } @Test public void testThreeKeyRangeSkip1() throws IOException, ExecutionException, InterruptedException { ColumnFamilyStore cfs = insertKey1Key2Key3(); IPartitioner p = StorageService.getPartitioner(); RangeSliceReply result = cfs.getRangeSlice(ArrayUtils.EMPTY_BYTE_ARRAY, Util.range(p, "key2", "key3"), 10, null, Arrays.asList("Column1".getBytes())); assertEquals(2, result.rows.size()); } Running this with "ant test" the partial output is [junit] Testsuite: org.apache.cassandra.db.ColumnFamilyStoreTest [junit] Tests run: 7, Failures: 2, Errors: 0, Time elapsed: 1.405 sec [junit] [junit] Testcase: testThreeKeyRangeAll(org.apache.cassandra.db.ColumnFamilyStoreTest): FAILED [junit] expected:<3> but was:<2> [junit] junit.framework.AssertionFailedError: expected:<3> but was:<2> [junit] at org .apache .cassandra .db .ColumnFamilyStoreTest.testThreeKeyRangeAll(ColumnFamilyStoreTest.java: 170) [junit] [junit] [junit] Testcase: testThreeKeyRangeSkip1(org.apache.cassandra.db.ColumnFamilyStoreTest): FAILED [junit] expected:<2> but was:<1> [junit] junit.framework.AssertionFailedError: expected:<2> but was:<1> [junit] at org .apache .cassandra .db .ColumnFamilyStoreTest .testThreeKeyRangeSkip1(ColumnFamilyStoreTest.java:184) [junit] [junit] [junit] Test org.apache.cassandra.db.ColumnFamilyStoreTest FAILED Any help appreciated. Aaron On 27 Apr 2010, at 09:38, aaron wrote: I've broken this case down further to some pyton code that works against the thrift generated client and am still getting the same odd results. With keys obejct1, object2 and object3 an open ended get_range_slice starting with "object1" only returns object1 and 2. I'm guessing that I've got something wrong or my expectation of how get_range_slice works is wrong, but I cannot see where I've gone wrong. Any help would be appreciated. They python code to add and read keys is below, assumes a Cassandra.Client connection. import time from cassandra import Cassandra,ttypes from thrift import Thrift from thrift.protocol import TBinaryProtocol from thrift.transport import TSocket, TTransport def add_data(conn): col_path = ttypes.ColumnPath(column_family="Standard1", column="col_name") consistency = ttypes.ConsistencyLevel.QUORUM for key in ["object1", "object2", "object3"]: conn.insert("Keyspace1", key, col_path, "col_value", int(time.time() * 1e6), consistency) return def read_range(conn, start_key, end_key): col_parent = ttypes.ColumnParent(column_family="Standard1") predicate = ttypes.SlicePredicate(column_names=["col_name"]) range = ttypes.KeyRange(start_key=start_key, end_key=end_key, count=1000) consistency = ttypes.ConsistencyLevel.QUORUM return conn.get_range_slices("Keyspace1", col_parent, predicate, range, consistency) Below is the result of calling read_range with different start values. I've also included the d
Bootstrap source code
Hi I have looking at the bootstrap source and seem to understand it for the most part. This is what I do not follow: (1) New node joins and doesn't advertise its token. (2) Requests nodes to send it data. The nodes that need to send it data first flush memtables and then transfer SSTables. Once the streaming is over the new node advertises its token and starts handling reads and writes correct? But what happens to keys that are being written to the old nodes after the memtables have been dumped. Looks like there is a window where keys would be going to the old nodes and not making it to the new nodes. Am I missing something here. -- Cheers Bill
Row slice / cache performance
1. I wrote the list a while back about less-than-great performance when reading thousands of columns even on cache hits. Last night, I decided to try to get to the bottom of why. I tested this by setting the row cache capacity on a TimeUUIDType-sorted CF to 10, filling up a single row with 2000 columns, and only running queries against that row. That row was the only thing in the database. I rm -Rf'd the data before starting the test. The tests were done from Coda Hale's scala client cassie, which is just a thin layer around the java thrift bindings. I didn't actually time each call because that wasn't the objective, but I didn't really need to. Reads of 10 columns felt quick enough, but 100 columns was slower. 1000 columns would frequently cause the client to timeout. The cache hit rate on that CF was 1.0, so, yes, the row was in cache. Doing a thousand reads with count=100 in a single thread pegged my macbook's CPU and caused the fans to spin up pretty loud. So, I attached a profiler and repeated the test. I'm no expert on cassandra internals, so please let me know if I'm way off here. The profiled reads were reversed=true, count=100. As far as I can tell, there are three components taking up most of the time on this type of read (row slice out of cache): 1. ColumnFamilystore.removeDeleted() @ ~40% - Most of the time in here is actually spent materializing UUID objects so that they can be compared in the ConcurrentSkipListMap (ColumnFamily.columns_). 2. SliceQueryFilter.getMemColumnIterator @ ~30% - Virtually all the time in here is spent in ConcurrentSkipListMap$Values.toArrray() 3. QueryFilter.collectCollatedColumns @ ~30% - All the time being spent in ColumnFamily.addColumn, and about half of the total spent materializing UUIDs for comparison. This profile is consistent with the decrease in performance with higher values for count. If there are more UUIDs to deserialize, the performance of removeDeleted(), and collectCollatedColumns() should increase (roughly) linearly. So, my question at this point is how to fix it. I have some basic ideas, but being new to cassandra internals, I'm not sure they make any sense. Help me out here: 1. Optionally call removeDeleted() less often. I realize that this is probably a bad idea for a lot of reasons, but it was the first thing I thought of. 2. When a ColumnFamily object is put in to the row cache, copy the columns over to another data structure that doesn't need to be sorted on get(). If columns_ needs to be kept around, this option would have a memory impact, but at least for us, it'd be well worth it for the speed. 3. I'd love to hear feedback on these / the rest of this (long) post.
Re: Row slice / cache performance
Not sure why the first paragraph turned in to a numbered bullet... On Sun, May 2, 2010 at 11:00 AM, James Golick wrote: > >1. I wrote the list a while back about less-than-great performance when >reading thousands of columns even on cache hits. Last night, I decided to >try to get to the bottom of why. > > > I tested this by setting the row cache capacity on a TimeUUIDType-sorted CF > to 10, filling up a single row with 2000 columns, and only running queries > against that row. That row was the only thing in the database. I rm -Rf'd > the data before starting the test. > > The tests were done from Coda Hale's scala client cassie, which is just a > thin layer around the java thrift bindings. I didn't actually time each call > because that wasn't the objective, but I didn't really need to. Reads of 10 > columns felt quick enough, but 100 columns was slower. 1000 columns would > frequently cause the client to timeout. The cache hit rate on that CF was > 1.0, so, yes, the row was in cache. > > Doing a thousand reads with count=100 in a single thread pegged my > macbook's CPU and caused the fans to spin up pretty loud. > > So, I attached a profiler and repeated the test. I'm no expert on cassandra > internals, so please let me know if I'm way off here. The profiled reads > were reversed=true, count=100. > > As far as I can tell, there are three components taking up most of the time > on this type of read (row slice out of cache): > >1. ColumnFamilystore.removeDeleted() @ ~40% - Most of the time in here >is actually spent materializing UUID objects so that they can be compared > in >the ConcurrentSkipListMap (ColumnFamily.columns_). >2. SliceQueryFilter.getMemColumnIterator @ ~30% - Virtually all the >time in here is spent in ConcurrentSkipListMap$Values.toArrray() >3. QueryFilter.collectCollatedColumns @ ~30% - All the time being spent >in ColumnFamily.addColumn, and about half of the total spent materializing >UUIDs for comparison. > > This profile is consistent with the decrease in performance with higher > values for count. If there are more UUIDs to deserialize, the performance of > removeDeleted(), and collectCollatedColumns() should increase (roughly) > linearly. > > So, my question at this point is how to fix it. I have some basic ideas, > but being new to cassandra internals, I'm not sure they make any sense. Help > me out here: > >1. Optionally call removeDeleted() less often. I realize that this is >probably a bad idea for a lot of reasons, but it was the first thing I >thought of. >2. When a ColumnFamily object is put in to the row cache, copy the >columns over to another data structure that doesn't need to be sorted on >get(). If columns_ needs to be kept around, this option would have a memory >impact, but at least for us, it'd be well worth it for the speed. >3. > > I'd love to hear feedback on these / the rest of this (long) post. >
Search Sample and Relation question because UDDI as Key
Hello, 1) Can you provide a solution or a sample for searching (Column and SuperColumn) (Fulltext). What is the Way to realize this? Hadoop/MapReduce? See you a posibility to build/use a index for columns? Why this: In a given Data-Model we "must" use UUIDs as Key and have actually no chance to seach values from "Columns"? (or not?) 2) How can we realize a "relation" For Sample: (http://arin.me/blog/wtf-is-a-supercolumn-cassandra-data-model) Arin describes good a simple Data-Model to build a Blog. But how can we read (filter) all Posts from "BlogEntries" from a single Autor? (filter the Supercolumns by a culum inside of a SuperColumn) The "relation" for Sample is Autor -> BlogEntries... To filter the Datas there is a needing to specify in a "get(...)"-Function a Column/Value combination... I know well that cassandra is not a "relational Database"! But without these releations the usage is very "limited" (specialized) Thanks in Advance! - and thx for Cassandra! With Hector i build a (Apache)Cocoon-Transformer... With Kind Regards, Falk Wolsky
How do you, Bloom filter of the false positive rate or remove the problem of distributed databases?
Hi This data structure recognizes to the way based on the idea of Eventually Consistency of BASE though Bloom filter is adopted for the data structure in Cassandra as shape to allow no limited adjustment. In a word, there is a problem of generating the false positive rate. Moreover, data is deleted as a common problem to an existing filesystem of OS and the distributed database including BigTable of Google. In the deletion of data, I think that I try to attempt solving by especially using Interval Tree Clocks of Vector Clock that is a kind of the logical clock of Lamport. So question. How in the Bloom filter to detect the false positive rate, or to resolve the problem? My guess is, Merkel Tree and I thought that Tombstone is concerned? PS. Cassandra has contributed to the Wiki's poor ability to Japanese translation. :-) --- Kazuki Aranami Twitter: http://twitter.com/kimtea Email: kazuki.aran...@gmail.com http://d.hatena.ne.jp/kazuki-aranami/ ---
Re: Row slice / cache performance
Just an update on this. I wrote a patch which attempts to solve this problem by keeping an index of columns that are marked for deletion to avoid having to iterate over the whole column set and call columns_.get() over and over again. My patch works, and the time spent in removeDeleted() is now close to zero. But, the performance doesn't seem to have noticeably improved. So, I'm not sure what I'm missing here. Either my test methodology is broken or I completely misread the profile. On Sun, May 2, 2010 at 11:01 AM, James Golick wrote: > Not sure why the first paragraph turned in to a numbered bullet... > > > On Sun, May 2, 2010 at 11:00 AM, James Golick wrote: > >> >>1. I wrote the list a while back about less-than-great performance >>when reading thousands of columns even on cache hits. Last night, I >> decided >>to try to get to the bottom of why. >> >> >> I tested this by setting the row cache capacity on a TimeUUIDType-sorted >> CF to 10, filling up a single row with 2000 columns, and only running >> queries against that row. That row was the only thing in the database. I rm >> -Rf'd the data before starting the test. >> >> The tests were done from Coda Hale's scala client cassie, which is just a >> thin layer around the java thrift bindings. I didn't actually time each call >> because that wasn't the objective, but I didn't really need to. Reads of 10 >> columns felt quick enough, but 100 columns was slower. 1000 columns would >> frequently cause the client to timeout. The cache hit rate on that CF was >> 1.0, so, yes, the row was in cache. >> >> Doing a thousand reads with count=100 in a single thread pegged my >> macbook's CPU and caused the fans to spin up pretty loud. >> >> So, I attached a profiler and repeated the test. I'm no expert on >> cassandra internals, so please let me know if I'm way off here. The profiled >> reads were reversed=true, count=100. >> >> As far as I can tell, there are three components taking up most of the >> time on this type of read (row slice out of cache): >> >>1. ColumnFamilystore.removeDeleted() @ ~40% - Most of the time in here >>is actually spent materializing UUID objects so that they can be compared >> in >>the ConcurrentSkipListMap (ColumnFamily.columns_). >>2. SliceQueryFilter.getMemColumnIterator @ ~30% - Virtually all the >>time in here is spent in ConcurrentSkipListMap$Values.toArrray() >>3. QueryFilter.collectCollatedColumns @ ~30% - All the time being >>spent in ColumnFamily.addColumn, and about half of the total spent >>materializing UUIDs for comparison. >> >> This profile is consistent with the decrease in performance with higher >> values for count. If there are more UUIDs to deserialize, the performance of >> removeDeleted(), and collectCollatedColumns() should increase (roughly) >> linearly. >> >> So, my question at this point is how to fix it. I have some basic ideas, >> but being new to cassandra internals, I'm not sure they make any sense. Help >> me out here: >> >>1. Optionally call removeDeleted() less often. I realize that this is >>probably a bad idea for a lot of reasons, but it was the first thing I >>thought of. >>2. When a ColumnFamily object is put in to the row cache, copy the >>columns over to another data structure that doesn't need to be sorted on >>get(). If columns_ needs to be kept around, this option would have a >> memory >>impact, but at least for us, it'd be well worth it for the speed. >>3. >> >> I'd love to hear feedback on these / the rest of this (long) post. >> > >
Re: Row slice / cache performance
Got a ~50% improvement by making UUID comparison less heavy-weight. https://issues.apache.org/jira/browse/CASSANDRA-1043 On Sun, May 2, 2010 at 7:49 PM, James Golick wrote: > Just an update on this. I wrote a patch which attempts to solve this > problem by keeping an index of columns that are marked for deletion to avoid > having to iterate over the whole column set and call columns_.get() over and > over again. > > My patch works, and the time spent in removeDeleted() is now close to zero. > But, the performance doesn't seem to have noticeably improved. So, I'm not > sure what I'm missing here. Either my test methodology is broken or I > completely misread the profile. > > On Sun, May 2, 2010 at 11:01 AM, James Golick wrote: > >> Not sure why the first paragraph turned in to a numbered bullet... >> >> >> On Sun, May 2, 2010 at 11:00 AM, James Golick wrote: >> >>> >>>1. I wrote the list a while back about less-than-great performance >>>when reading thousands of columns even on cache hits. Last night, I >>> decided >>>to try to get to the bottom of why. >>> >>> >>> I tested this by setting the row cache capacity on a TimeUUIDType-sorted >>> CF to 10, filling up a single row with 2000 columns, and only running >>> queries against that row. That row was the only thing in the database. I rm >>> -Rf'd the data before starting the test. >>> >>> The tests were done from Coda Hale's scala client cassie, which is just a >>> thin layer around the java thrift bindings. I didn't actually time each call >>> because that wasn't the objective, but I didn't really need to. Reads of 10 >>> columns felt quick enough, but 100 columns was slower. 1000 columns would >>> frequently cause the client to timeout. The cache hit rate on that CF was >>> 1.0, so, yes, the row was in cache. >>> >>> Doing a thousand reads with count=100 in a single thread pegged my >>> macbook's CPU and caused the fans to spin up pretty loud. >>> >>> So, I attached a profiler and repeated the test. I'm no expert on >>> cassandra internals, so please let me know if I'm way off here. The profiled >>> reads were reversed=true, count=100. >>> >>> As far as I can tell, there are three components taking up most of the >>> time on this type of read (row slice out of cache): >>> >>>1. ColumnFamilystore.removeDeleted() @ ~40% - Most of the time in >>>here is actually spent materializing UUID objects so that they can be >>>compared in the ConcurrentSkipListMap (ColumnFamily.columns_). >>>2. SliceQueryFilter.getMemColumnIterator @ ~30% - Virtually all the >>>time in here is spent in ConcurrentSkipListMap$Values.toArrray() >>>3. QueryFilter.collectCollatedColumns @ ~30% - All the time being >>>spent in ColumnFamily.addColumn, and about half of the total spent >>>materializing UUIDs for comparison. >>> >>> This profile is consistent with the decrease in performance with higher >>> values for count. If there are more UUIDs to deserialize, the performance of >>> removeDeleted(), and collectCollatedColumns() should increase (roughly) >>> linearly. >>> >>> So, my question at this point is how to fix it. I have some basic ideas, >>> but being new to cassandra internals, I'm not sure they make any sense. Help >>> me out here: >>> >>>1. Optionally call removeDeleted() less often. I realize that this is >>>probably a bad idea for a lot of reasons, but it was the first thing I >>>thought of. >>>2. When a ColumnFamily object is put in to the row cache, copy the >>>columns over to another data structure that doesn't need to be sorted on >>>get(). If columns_ needs to be kept around, this option would have a >>> memory >>>impact, but at least for us, it'd be well worth it for the speed. >>>3. >>> >>> I'd love to hear feedback on these / the rest of this (long) post. >>> >> >> >
Re: inserting new rows with one key vs. inserting new columns in a row performance
Hello, It seems that I have experienced network problems (local pre-installed firewall) and some rest http inefficiencies, so I think that it behaves the same in both cases. I am sorry to have taken from your time. Best regards, Daniel. На 30 април 2010 20:46, Даниел Симеонов написа: > Hi, >I've checked two similar scenarios and one of them seem to be more > performant. So timestamped data is being appended, the first use case is > with an OPP and new rows being created every with only one column (there are > about 7-8 CFs). The second cases is to have rows with more columns and > RandomPartitioner, although every row gets much more than one column > appended yet the inserts are relatively uniformly distributed among rows. > Yet the first scenario is faster than the second, and the second one starts > with good response times (about 20-30 ms) and gradually the mean time > increases (to about 150-200 ms). What could be the reason? > Thank you very much! > Best regards, Daniel. >