Re: cqlinputformat and retired cqlpagingingputformat creates lots of connections to query the server
Shenghua, The problem is the user might only want all the data via a select * like statement. It seems that 257 connections to query the rows are necessary. However, is there any way to prohibit 257 concurrent connections? Your reasoning is correct. The number of connections should be tunable via the cassandra.input.split.size property. See ConfigHelper.setInputSplitSize(..) The problem is that vnodes completely trashes this, since splits returned don't span across vnodes. There's an issue out for this – https://issues.apache.org/jira/browse/CASSANDRA-6091 but part of the problem is that the thrift stuff involved here is getting rewritten¹ to be pure cql. In the meantime you override the CqlInputFormat and manually re-merge splits together, where location sets match, so to better honour inputSplitSize and to return to a more reasonable number of connections. We do this, using code similar to this patch https://github.com/michaelsembwever/cassandra/pull/2/files ~mck ¹ https://issues.apache.org/jira/browse/CASSANDRA-8358
Re: cqlinputformat and retired cqlpagingingputformat creates lots of connections to query the server
I did another experiment to verify indeed 3*257 (1 of 257 ranges is null effectively) mappers were created. Thanks mcm for the information ! On Wed, Jan 28, 2015 at 12:17 AM, mck m...@apache.org wrote: Shenghua, The problem is the user might only want all the data via a select * like statement. It seems that 257 connections to query the rows are necessary. However, is there any way to prohibit 257 concurrent connections? Your reasoning is correct. The number of connections should be tunable via the cassandra.input.split.size property. See ConfigHelper.setInputSplitSize(..) The problem is that vnodes completely trashes this, since splits returned don't span across vnodes. There's an issue out for this – https://issues.apache.org/jira/browse/CASSANDRA-6091 but part of the problem is that the thrift stuff involved here is getting rewritten¹ to be pure cql. In the meantime you override the CqlInputFormat and manually re-merge splits together, where location sets match, so to better honour inputSplitSize and to return to a more reasonable number of connections. We do this, using code similar to this patch https://github.com/michaelsembwever/cassandra/pull/2/files ~mck ¹ https://issues.apache.org/jira/browse/CASSANDRA-8358 -- Regards, Shenghua (Daniel) Wan
Re: cqlinputformat and retired cqlpagingingputformat creates lots of connections to query the server
If you are using replication factor 1 and 3 cassandra nodes, 256 virtual nodes should be evenly distributed on 3 nodes. So there are totally 256 virtual nodes. But in your experiment, you saw 3*257 mapper. Is that because of the setting cassandra.input.split.size=3? It is nothing with node number=3. Otherwise, I am confused why there are 256 virtual nodes on every cassandra node. On Wed, Jan 28, 2015 at 12:29 AM, Shenghua(Daniel) Wan wansheng...@gmail.com wrote: I did another experiment to verify indeed 3*257 (1 of 257 ranges is null effectively) mappers were created. Thanks mcm for the information ! On Wed, Jan 28, 2015 at 12:17 AM, mck m...@apache.org wrote: Shenghua, The problem is the user might only want all the data via a select * like statement. It seems that 257 connections to query the rows are necessary. However, is there any way to prohibit 257 concurrent connections? Your reasoning is correct. The number of connections should be tunable via the cassandra.input.split.size property. See ConfigHelper.setInputSplitSize(..) The problem is that vnodes completely trashes this, since splits returned don't span across vnodes. There's an issue out for this – https://issues.apache.org/jira/browse/CASSANDRA-6091 but part of the problem is that the thrift stuff involved here is getting rewritten¹ to be pure cql. In the meantime you override the CqlInputFormat and manually re-merge splits together, where location sets match, so to better honour inputSplitSize and to return to a more reasonable number of connections. We do this, using code similar to this patch https://github.com/michaelsembwever/cassandra/pull/2/files ~mck ¹ https://issues.apache.org/jira/browse/CASSANDRA-8358 -- Regards, Shenghua (Daniel) Wan
Re: cqlinputformat and retired cqlpagingingputformat creates lots of connections to query the server
That's c* default setting. My version is 2.0.11. Check your Cassandra.yaml. On Jan 28, 2015 4:53 PM, Huiliang Zhang zhl...@gmail.com wrote: If you are using replication factor 1 and 3 cassandra nodes, 256 virtual nodes should be evenly distributed on 3 nodes. So there are totally 256 virtual nodes. But in your experiment, you saw 3*257 mapper. Is that because of the setting cassandra.input.split.size=3? It is nothing with node number=3. Otherwise, I am confused why there are 256 virtual nodes on every cassandra node. On Wed, Jan 28, 2015 at 12:29 AM, Shenghua(Daniel) Wan wansheng...@gmail.com wrote: I did another experiment to verify indeed 3*257 (1 of 257 ranges is null effectively) mappers were created. Thanks mcm for the information ! On Wed, Jan 28, 2015 at 12:17 AM, mck m...@apache.org wrote: Shenghua, The problem is the user might only want all the data via a select * like statement. It seems that 257 connections to query the rows are necessary. However, is there any way to prohibit 257 concurrent connections? Your reasoning is correct. The number of connections should be tunable via the cassandra.input.split.size property. See ConfigHelper.setInputSplitSize(..) The problem is that vnodes completely trashes this, since splits returned don't span across vnodes. There's an issue out for this – https://issues.apache.org/jira/browse/CASSANDRA-6091 but part of the problem is that the thrift stuff involved here is getting rewritten¹ to be pure cql. In the meantime you override the CqlInputFormat and manually re-merge splits together, where location sets match, so to better honour inputSplitSize and to return to a more reasonable number of connections. We do this, using code similar to this patch https://github.com/michaelsembwever/cassandra/pull/2/files ~mck ¹ https://issues.apache.org/jira/browse/CASSANDRA-8358 -- Regards, Shenghua (Daniel) Wan
error while bulk loading using copy command
Hi All, We need to upload 18 lacs rows into a table which consist columns with data type counter. on uploading using copy command , we are getting below error: *Bad Request: INSERT statement are not allowed on counter tables, use UPDATE instead* we need counter data type because after loading this data we want to use functionality of counter data type. Kindly help is there any way to do this. Regards: Rahul Bhardwaj -- Follow IndiaMART.com http://www.indiamart.com for latest updates on this and more: https://plus.google.com/+indiamart https://www.facebook.com/IndiaMART https://twitter.com/IndiaMART Mobile Channel: https://itunes.apple.com/WebObjects/MZStore.woa/wa/viewSoftware?id=668561641mt=8 https://play.google.com/store/apps/details?id=com.indiamart.m http://m.indiamart.com/ https://www.youtube.com/watch?v=DzORNbeSXN8list=PL2o4J51MqpL0mbue6kzDa6eymLVUXtlR1index=2 Watch how Irrfan Khan gets his work done in no time on IndiaMART, kyunki Kaam Yahin Banta Hai https://www.youtube.com/watch?v=hmS4Afl2bNU!!!
Re: incremential repairs - again
Hi, the automatically meant this reply earlier: If you are on 2.1.2+ (or using STCS) you don't those steps (should probably update the blog post). Now we keep separate levelings for the repaired/unrepaired data and move the sstables over after the first incremental repair My understanding was, that there are no steps necessary to migrate to incremental repairs (disableautocompaction and so on) for 2.1.2+. I'm looking forward to the next release - and I'll give a test run. Thanks so far :-)
incremential repairs - again
Hi, a short question about the new incremental repairs again. I am running 2.1.2 (for testing). Marcus pointed me that 2.1.2 should do incremental repairs automatically, so I rolled back all steps taken. I expect that routine repair times will decrease when I do not put many new data on the cluster. But they dont - they are constant at about 1000 minutes per node, so I extracted all Repaired at with sstablemetadata and I cant see any recent date. I put several GB of data into the cluster in 2015 and I run nodetool repair -pr on every node regularly. Am I still missing something? Or is this one of the issues with 2.1.2 (CASSANDRA-8316)? Thanks for hints, Jan
Re: incremential repairs - again
Hi Unsure what you mean by automatically, but you should use -par -inc when you repair And, you should wait until 2.1.3 (which will be out very soon) before doing this, we have fixed many issues with incremental repairs /Marcus On Thu, Jan 29, 2015 at 7:44 AM, Roland Etzenhammer r.etzenham...@t-online.de wrote: Hi, a short question about the new incremental repairs again. I am running 2.1.2 (for testing). Marcus pointed me that 2.1.2 should do incremental repairs automatically, so I rolled back all steps taken. I expect that routine repair times will decrease when I do not put many new data on the cluster. But they dont - they are constant at about 1000 minutes per node, so I extracted all Repaired at with sstablemetadata and I cant see any recent date. I put several GB of data into the cluster in 2015 and I run nodetool repair -pr on every node regularly. Am I still missing something? Or is this one of the issues with 2.1.2 (CASSANDRA-8316)? Thanks for hints, Jan
Re: full-tabe scan - extracting all data from C*
Hint: using the Java driver, you can set the fetchSize to tell the driver how many CQL rows to fetch for each page. Depending on the size (in bytes) of each CQL row, it would be useful to tune this fetchSize value to avoid loading too much data into memory for each page On Wed, Jan 28, 2015 at 8:20 AM, Xu Zhongxing xu_zhong_x...@163.com wrote: This is hard to answer. The performance is a thing depending on context. You could tune various parameters. At 2015-01-28 14:43:38, Shenghua(Daniel) Wan wansheng...@gmail.com wrote: Cool. What about performance? e.g. how many record for how long? On Tue, Jan 27, 2015 at 10:16 PM, Xu Zhongxing xu_zhong_x...@163.com wrote: For Java driver, there is no special API actually, just ResultSet rs = session.execute(select * from ...); for (Row r : rs) { ... } For Spark, the code skeleton is: val rdd = sc.cassandraTable(ks, table) then call various standard Spark API to process the table parallelly. I have not used CqlInputFormat. At 2015-01-28 13:38:20, Shenghua(Daniel) Wan wansheng...@gmail.com wrote: Hi, Zhongxing, I am also interested in your table size. I am trying to dump 10s Million record data from C* using map-reduce related API like CqlInputFormat. You mentioned about Java driver. Could you suggest any API you used? Thanks. On Tue, Jan 27, 2015 at 5:33 PM, Xu Zhongxing xu_zhong_x...@163.com wrote: Both Java driver select * from table and Spark sc.cassandraTable() work well. I use both of them frequently. At 2015-01-28 04:06:20, Mohammed Guller moham...@glassbeam.com wrote: Hi – Over the last few weeks, I have seen several emails on this mailing list from people trying to extract all data from C*, so that they can import that data into other analytical tools that provide much richer analytics functionality than C*. Extracting all data from C* is a full-table scan, which is not the ideal use case for C*. However, people don’t have much choice if they want to do ad-hoc analytics on the data in C*. Unfortunately, I don’t think C* comes with any built-in tools that make this task easy for a large dataset. Please correct me if I am wrong. Cqlsh has a COPY TO command, but it doesn’t really work if you have a large amount of data in C*. I am aware of couple of approaches for extracting all data from a table in C*: 1) Iterate through all the C* partitions (physical rows) using the Java Driver and CQL. 2) Extract the data directly from SSTables files. Either approach can be used with Hadoop or Spark to speed up the extraction process. I wanted to do a quick survey and find out how many people on this mailing list have successfully used approach #1 or #2 for extracting large datasets (terabytes) from C*. Also, if you have used some other techniques, it would be great if you could share your approach with the group. Mohammed -- Regards, Shenghua (Daniel) Wan -- Regards, Shenghua (Daniel) Wan