Corrupt SSTable after dropping column
Hi, It seems like dropping a column can cause a "java.io.IOException: Corrupt empty row found in unfiltered partition" exception when existing SSTables are later compacted. This seems to happen with all Cassandra 3.x versions and is very easy to replicate. I've created a jira with all the details: https://issues.apache.org/jira/browse/CASSANDRA-13337 This is extra problematic with Cassandra < 3.10 where nodes will fail to start once this happens. Have anyone else seen this? / Jonas
Re: In place vnode conversion possible?
On 18/12/14 21:45, Robert Coli wrote: > On Tue, Dec 16, 2014 at 12:38 AM, Jonas Borgström <mailto:jo...@borgstrom.se>> wrote: > > That said, I've done some testing and it appears to be possible to > perform an in place conversion as long as all nodes contain all data (3 > nodes and replication factor 3 for example) like this: > > > I would expect this to work, but to stream up to RF x the data around. Why would any streaming take place? Simply changing the tokens and restarting a node does not seem to trigger any streaming. And if I manually trigger a "nodetool repair" I notice almost no streaming since all nodes were already responsible for 100% of the data (RF = NUM_NODES). / Jonas signature.asc Description: OpenPGP digital signature
In place vnode conversion possible?
Hi, I know that adding a new vnode enabled DC is the recommended method to convert and existing cluster to vnode. And that the cassandra-shuffle utility has been removed. That said, I've done some testing and it appears to be possible to perform an in place conversion as long as all nodes contain all data (3 nodes and replication factor 3 for example) like this: for each node: - nodetool -h localhost disablegossip (Not sure if this is needed) - cqlsh localhost UPDATE system.local SET tokens=$NEWTOKENS WHERE key='local'; - nodetool -h localhost disablethrift (Not sure if this is needed) - nodetool -h localhost drain - service cassandra restart And the following python snippet was used to generate $NEWTOKENS for each node (RandomPartitioner): """ import random print str([str(x) for x in sorted(random.randint(0,2**127-1) for x in range(256))]).replace('[', '{').replace(']', '}') """ I've tested this in a test cluster and it seems to work just fine. Has anyone else done anything similar? Or if manually changing tokens is impossible and something horrible will hit me down the line? Test cluster configuration -- Cassandra version: 1.2.19 Number of nodes: 3 Keyspace: NetworkTopologyStrategy: {DC1: 1, DC2:1, DC3: 1} / Jonas signature.asc Description: OpenPGP digital signature
Re: [RELEASE] Apache Cassandra 1.0.10 released
Hi, Can someone give some more details about the CASSANDRA-4116 bug fixed in this release? Could this cause resurrection of deleted data for example? https://issues.apache.org/jira/browse/CASSANDRA-4116 / Jonas On 2012-05-08 11:04 , Sylvain Lebresne wrote: > The Cassandra team is pleased to announce the release of Apache Cassandra > version 1.0.10. > > Cassandra is a highly scalable second-generation distributed database, > bringing together Dynamo's fully distributed design and Bigtable's > ColumnFamily-based data model. You can read more here: > > http://cassandra.apache.org/ > > Downloads of source and binary distributions are listed in our download > section: > > http://cassandra.apache.org/download/ > > This version is maintenance/bug fix release[1]. As always, please pay > attention to the release notes[2] and Let us know[3] if you were to encounter > any problem. > > Have fun! > > [1]: http://goo.gl/u8gIO (CHANGES.txt) > [2]: http://goo.gl/mAHbY (NEWS.txt) > [3]: https://issues.apache.org/jira/browse/CASSANDRA signature.asc Description: OpenPGP digital signature
Re: sstable2json and resurrected rows
On 2012-03-31 08:45 , Zhu Han wrote: Did you hit the bug here? https://issues.apache.org/jira/browse/CASSANDRA-4054 Yes looks like it. But what confuses me most is not the sstable2json bug but why the major compaction does not replace the deleted row data with a tombstone. Is that a bug or a feature? To me it just looks like a wast of disk space... / Jonas best regards, 坚果云 <https://jianguopuzi.com/>, 最简捷易用的云存储 无限空间, 文件同步, 备份和分享! 2012/3/30 Jonas Borgström mailto:jo...@borgstrom.se>> Let me rephrase my question: Is it true that deleted rows will still be present in the sstable after a major compaction with 1.0.8 (not just tombstones)? Or did I mess up my test below? / Jonas On 2012-03-28 10:23 , Jonas Borgström wrote: Hi all, I've noticed a change in behavior between 0.8.10 and 1.0.8 when it comes to sstable2json output and major compactions. Is this a bug or intended behavior? With 1.0.8: create keyspace ks; use ks; create column family foo; set foo[1][1] = 1; nodetool -h localhost flush sstable2json foo-hc-1-Data.db => { "01": [["01","01",1332920802272000]] } del foo[1]; set foo[2][2] = 2; nodetool -h localhost flush sstable2json foo-hc-2-Data.db => { "01": [], "02": [["02","02",133292084309]] } nodetool -h localhost compact ks foo So far so good. But now I expect the resulting sstable to look like foo-hc-2 (the way 0.8.10 behaves) but instead it looks like the deleted foo[1] has been resurrected (foo[1] is still deleted when using the thrift api): sstable2json foo-hc-3-Data.db => { "01": [["01","01",1332920802272000]]__, "02": [["02","02",133292084309]] } So why is the full foo[1] row included in the sstable2json output and not just a tombstone? This is both a wast of disk space and makes it impossible to trust the sstable2json output. / Jonas
Re: sstable2json and resurrected rows
Let me rephrase my question: Is it true that deleted rows will still be present in the sstable after a major compaction with 1.0.8 (not just tombstones)? Or did I mess up my test below? / Jonas On 2012-03-28 10:23 , Jonas Borgström wrote: Hi all, I've noticed a change in behavior between 0.8.10 and 1.0.8 when it comes to sstable2json output and major compactions. Is this a bug or intended behavior? With 1.0.8: create keyspace ks; use ks; create column family foo; set foo[1][1] = 1; nodetool -h localhost flush sstable2json foo-hc-1-Data.db => { "01": [["01","01",1332920802272000]] } del foo[1]; set foo[2][2] = 2; nodetool -h localhost flush sstable2json foo-hc-2-Data.db => { "01": [], "02": [["02","02",133292084309]] } nodetool -h localhost compact ks foo So far so good. But now I expect the resulting sstable to look like foo-hc-2 (the way 0.8.10 behaves) but instead it looks like the deleted foo[1] has been resurrected (foo[1] is still deleted when using the thrift api): sstable2json foo-hc-3-Data.db => { "01": [["01","01",1332920802272000]], "02": [["02","02",133292084309]] } So why is the full foo[1] row included in the sstable2json output and not just a tombstone? This is both a wast of disk space and makes it impossible to trust the sstable2json output. / Jonas
sstable2json and resurrected rows
Hi all, I've noticed a change in behavior between 0.8.10 and 1.0.8 when it comes to sstable2json output and major compactions. Is this a bug or intended behavior? With 1.0.8: create keyspace ks; use ks; create column family foo; set foo[1][1] = 1; nodetool -h localhost flush sstable2json foo-hc-1-Data.db => { "01": [["01","01",1332920802272000]] } del foo[1]; set foo[2][2] = 2; nodetool -h localhost flush sstable2json foo-hc-2-Data.db => { "01": [], "02": [["02","02",133292084309]] } nodetool -h localhost compact ks foo So far so good. But now I expect the resulting sstable to look like foo-hc-2 (the way 0.8.10 behaves) but instead it looks like the deleted foo[1] has been resurrected (foo[1] is still deleted when using the thrift api): sstable2json foo-hc-3-Data.db => { "01": [["01","01",1332920802272000]], "02": [["02","02",133292084309]] } So why is the full foo[1] row included in the sstable2json output and not just a tombstone? This is both a wast of disk space and makes it impossible to trust the sstable2json output. / Jonas
Minor version upgrade and SSTable compatibilty
Hi, I have some questions about SSTable compatibility when doing a minor version upgrade. For example when upgrading from 1.0.3 (Uses version "hb") to 1.0.6 (uses version "hc"). 1. Will I need to upgrade all nodes before performing streaming/repair? 2. Will it be possible to downgrade a node from 1.0.6 to 1.0.3 and keep newly created SSTables? Or will I then have to revert to a pre-upgrade snapshot? / Jonas
Re: Previously deleted rows resurrected by repair?
On 2011-12-28 12:52 , Dominic Williams wrote: Hmm interesting could be some variation on 3510 (which caught me out). Actually after making some further reading of the changelog 2786 looks like a likely culprit. If I'm reading the jira correctly all versions < 0.8.8 and < 1.0.4 are at risk of getting deleted rows resurrected. Hopefully an upgrade to 1.0.6 will stop this problem from increasing but I still need to manually re-delete a bunch of rows. https://issues.apache.org/jira/browse/CASSANDRA-2786 Personally I really don't like having to rely on repair to stop deletes being undone. If you agree follow this proposal for an alternative https://issues.apache.org/jira/browse/CASSANDRA-3620 which also stops tombstone build up. Thanks, this definitely looks interesting. I'll have a look. / Jonas
Previously deleted rows resurrected by repair?
Hi, I Have a 3 node cluster running Cassandra 1.0.3 and using replication factor=3. Recently I've noticed that some previously deleted rows have started to reappear for some reason. And now I wonder if this is a known issue with 1.0.3? Repairs have been running every weekend (gc_grace is 10 days) and always completed successfully. But while looking at the logs I noticed that a fair number of ranges (around 10% of the total number of keys) have been streamed between these nodes during the repair sessions. This seems a bit high to me given that everything is written using quorum and all nodes have been up all the time. For me this looks suspiciously like some already deleted keys are streamed to other nodes during repair. Some more details about the data: All keys are written to only once and most of them are deleted a couple of days/weeks later. Some keys are large enough to require incremental compaction. Could this bug cause this? https://issues.apache.org/jira/browse/CASSANDRA-3510 Regards, Jonas
Proposed 1.0.3: StackOverflowError on repair
Hi While testing the proposed 1.0.3 version I got the following exception while running repair: (StackOverflowError) http://pastebin.com/raw.php?i=35Rt7ryB The affected column family is denfined like this: create column family FileStore with comparator=UTF8Type and key_validation_class = 'UTF8Type' and column_metadata=[{column_name: namespace, validation_class: UTF8Type, index_type: KEYS}, {column_name: version, validation_class: IntegerType}, {column_name: chunk_size, validation_class: IntegerType}, {column_name: num_chunks, validation_class: IntegerType}, {column_name: size, validation_class: LongType}]; Is this a known issue? / Jonas
Re: Moving to a new cluster
On 09/22/2011 01:25 AM, aaron morton wrote: *snip* > When you start a repair it will repair will the other nodes it > replicates data with. So you only need to run it every RF nodes. Start > it one one, watch the logs to see who it talks to and then start it on > the first node it does not talk to. And so on. Is this new in 0.8 or has it always been this way? From http://wiki.apache.org/cassandra/Operations#Frequency_of_nodetool_repair """ Unless your application performs no deletes, it is vital that production clusters run nodetool repair periodically on all nodes in the cluster. """ So for a 3 node cluster using RF=3, is it sufficient to run "nodetool repair" on one node? / Jonas
Re: cassandra crashed while repairing, leave node size X3
On 09/19/2011 04:26 AM, Anand Somani wrote: > In my tests I have seen repair sometimes take a lot of space (2-3 > times), cleanup did not clean it, the only way I could clean that was > using major compaction. Do you remember with what version you saw these problems? I've had the same problems with 0.7.4 but so far my repair tests with 0.8.6 seems to behave a lot better. / Jonas
Re: Error in upgrading cassandra to 0.8.5
On 09/13/2011 05:21 PM, Jonathan Ellis wrote: > More or less. NEWS.txt explains upgrade procedure in more detail. When moving from 0.7.x to 0.8.5 do I need to scrub all sstables post upgrade? NEWS.txt doesn't mention anything about that but your comment here seems to indicate so: https://issues.apache.org/jira/browse/CASSANDRA-2739?focusedCommentId=13071490&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13071490 / Jonas
Re: AW: Strange nodetool repair behaviour
On 04/05/2011 03:49 PM, Jonathan Ellis wrote: > Sounds like https://issues.apache.org/jira/browse/CASSANDRA-2324 Yes, that sounds like the issue I'm having. Any chance for a fix for this being backported to 0.7.x? Anyway, I guess I might as well share the test case I've used to reproduce this problem: Cluster configuration: 6 nodes running 0.7.4 with RF=3 1. Create keyspace and column families (see repair_test.py (attached)) 2. Insert 20 100MB keys into each of column family A, B and C: $ python repair_test.py This results in 2.4GB worth of sstables on node1: $ du -sh /data/cassandra/data/repair_test3/ 2.4G/data/cassandra/data/repair_test3/ 3. Run repair: $ time nodetool -h node1 repair repair_test3 real3m28.218s The repair logged about streaming of 1 to 3 ranges for each column family and the sstable directory was filled with a bunch of "-tmp-" files and disk usage peaked at 10+GB The repair completed successfully and the disk usage is down to 6.4GB: $ du -sh /data/cassandra/data/repair_test3/ 6.4G/data/cassandra/data/repair_test3/ 4. Run repair again: $ time nodetool -h node1 repair repair_test3 real9m23.514s This time the disk usage peaked at 25+GB and then settled at 4.7GB. This time repair reported that even more ranges were out of sync. So this issue seems to cause repair to take a very long time, unnecessarily sending a lot of data over the network and leave a lot of "air" in the resulting sstables that can only be recovered by triggering major compactions. (A GC was triggered before all disk usage measurements) Regards, Jonas import pycassa """ create keyspace repair_test3 with replication_factor=3; use repair_test3; create column family A with memtable_throughput=32; create column family B with memtable_throughput=32; create column family C with memtable_throughput=32; """ servers = ['node1:9160', 'node2:9160', 'node3:9160', 'node4:9160', 'node5:9160', 'node6:9160'] def insert_data(cf_name): pool = pycassa.ConnectionPool('repair_test3', servers) cf = pycassa.ColumnFamily(pool, cf_name, write_consistency_level=pycassa.ConsistencyLevel.ONE) data = 'X' * 1024*1024 for x in range(20): for y in range(100): print cf_name, x, y cf.insert(str(x), {str(y): data}) insert_data('A') insert_data('B') insert_data('C')
Strange nodetool repair behaviour
Hi, I have a 6 node 0.7.4 cluster with replication_factor=3 where "nodetool repair keyspace" behaves really strange. The keyspace contains three column families and about 60GB data in total (i.e 30GB on each node). Even though no data has been added or deleted since the last repair, a repair takes hours and the repairing node seems to receive 100+GB worth of sstable data from its neighbourhood nodes, i.e several times the actual data size. The log says things like: "Performing streaming repair of 27 ranges" And a bunch of: "Compacted to 22,208,983,964 to 4,816,514,033 (~21% of original)" In the end the repair finishes without any error after a few hours but even then the active sstables seems to contain lots of redundant data since the disk usage can be sliced in half by triggering a major compaction. All this leads me to believe that something stops the AES from correctly figuring out what data is already on the repairing node and what needs to be streamed from the neighbours. The only thing I can think of right now is that one of the column families contains a lot of large rows that are larger than memtable_throughput and that's perhaps what's confusing the merkle tree. Anyway, is this a known problem of perhaps expected behaviour? Otherwise I'll try to create a more reproducible test case. Regards, Jonas
Re: Possible EOFException regression in 0.7.1
On 02/16/2011 03:54 PM, Jonathan Ellis wrote: > It does look a lot like 1932. Make sure everything is really running > 0.7.2, 0.7.0 can't read data files created by 0.7.1+. All nodes are running 0.7.2 > If the versions are ok, take a snapshot, then compact, and see if the > problem still occurs with all files on the [compacted] new format. Yes I can confirm that rewriting all sstables by issuing "compact" did the trick. Before I started I noticed that some nodes had both e and f sstables and some only f files. So I guess the problem is that the version f files generated by 0.7.1 sometimes contained corrupted column name bloom filters somehow. Anyway, "compact" did make the problem go away. Thanks! / Jonas
Re: Possible EOFException regression in 0.7.1
gIterator.next(CollatingIterator.java:230) at org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) at org.apache.cassandra.db.RowIterator.hasNext(RowIterator.java:49) at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1354) at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:49) ... 4 more The column family was created with 0.7.0 and data has been inserted with 0.7.0, 0.7.1 and 0.7.2. There seems to be sstable files of version e and f. There's probably around 1000-2000 keys and each key has up to 1 million columns. Does this mean that the EOFException bug could have caused corrupted bloom filters after all or is this perhaps a separate issue, perhaps: https://issues.apache.org/jira/browse/CASSANDRA-1932 ? Regards, Jonas > Hoping this is quick enough. > > > > 2011/2/15 Jonathan Ellis mailto:jbel...@gmail.com>> > > I can reproduce with your script. Thanks! > > 2011/2/15 Jonas Borgström <mailto:jonas.borgst...@trioptima.com>>: > > Hi all, > > > > While testing the new 0.7.1 release I got the following exception: > > > > ERROR [ReadStage:11] 2011-02-15 16:39:18,105 > > DebuggableThreadPoolExecutor.java (line 103) Error in > ThreadPoolExecutor > > java.io.IOError: java.io.EOFException > >at > > > > org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTableNamesIterator.java:75) > >at > > > > org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:59) > >at > > > > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80) > >at > > > > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1274) > >at > > > > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1166) > >at > > > > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1095) > >at org.apache.cassandra.db.Table.getRow(Table.java:384) > >at > > > > org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:60) > >at > > > > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:473) > >at > > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > >at > > > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) > >at > > > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) > >at java.lang.Thread.run(Thread.java:636) > > Caused by: java.io.EOFException > >at > java.io.DataInputStream.readInt(DataInputStream.java:392) > >at > > > > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:48) > >at > > > > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30) > >at > > > > org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108) > >at > > > > org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106) > >at > > > > org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTableNamesIterator.java:71) > >... 12 more > > > > I'm able reliably reproduce this using the following one node > cluster: > > - apache-cassandra-0.7.1-bin.tar.gz > > - Fedora 14 > > - java version "1.6.0_20". > > OpenJDK 64-Bit Server VM (build 19.0-b09, mixed mode) > > - Default cassandra.yaml > > - cassandra-env.sh: MAX_HEAP_SIZE="1G"; HEAP_NEWSIZE="200M" > > > > cassandra-c
Possible EOFException regression in 0.7.1
Hi all, While testing the new 0.7.1 release I got the following exception: ERROR [ReadStage:11] 2011-02-15 16:39:18,105 DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecutor java.io.IOError: java.io.EOFException at org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTableNamesIterator.java:75) at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:59) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1274) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1166) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1095) at org.apache.cassandra.db.Table.getRow(Table.java:384) at org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:60) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:473) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) Caused by: java.io.EOFException at java.io.DataInputStream.readInt(DataInputStream.java:392) at org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:48) at org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30) at org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTableNamesIterator.java:71) ... 12 more I'm able reliably reproduce this using the following one node cluster: - apache-cassandra-0.7.1-bin.tar.gz - Fedora 14 - java version "1.6.0_20". OpenJDK 64-Bit Server VM (build 19.0-b09, mixed mode) - Default cassandra.yaml - cassandra-env.sh: MAX_HEAP_SIZE="1G"; HEAP_NEWSIZE="200M" cassandra-cli initialization: - create keyspace foo; - use foo; - create column family datasets; $ python dataset_check.py (attached) Inserting row 0 of 10 Inserting row 1 of 10 Inserting row 2 of 10 Inserting row 3 of 10 Inserting row 4 of 10 Inserting row 5 of 10 Inserting row 6 of 10 Inserting row 7 of 10 Inserting row 8 of 10 Inserting row 9 of 10 Attempting to fetch key 0 Traceback (most recent call last): ... pycassa.pool.MaximumRetryException: Retried 6 times After this I have 6 EOFExceptions in system.log. Running "get datasets[0]['name'];" using cassandra-cli also triggers the same exception. I've not been able to reproduce this with cassandra 0.7.0. Regards, Jonas import pycassa pool = pycassa.ConnectionPool('foo', ['localhost:9160'], timeout=10) cf = pycassa.ColumnFamily(pool, 'datasets') def insert_dataset(key, num_cols=5): columns = {} extra_data = 'XXX' * 20 for i in range(num_cols): col = 'r%08d' % i columns[col] = '%s:%s:%s' % (key, col, extra_data) if len(columns) >= 3000: cf.insert(key, columns) columns = {} if len(columns) >= 3000: cf.insert(key, columns) columns = {} cf.insert(key, {'name': 'key:%s' % key}) def test_insert_and_column_fetch(num=20): # Insert @num fairly large rows for i in range(num): print 'Inserting row %d of %d' % (i, num) insert_dataset(str(i)) # Verify that the "name" column is correctly stored for i in range(num): print 'Attempting to fetch key %d' % i row = cf.get(str(i), columns=['name']) assert row['name'] == 'key:%d' % i for i, (key, row) in enumerate(cf.get_range(columns=['name'])): print '%d: get_range returned: key %s, name: "%s"' % (i, key, row['name']) assert row['name'] == 'key:' + key test_insert_and_column_fetch(10)