Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BF778F32E for ; Tue, 23 Apr 2013 17:36:15 +0000 (UTC) Received: (qmail 54349 invoked by uid 500); 23 Apr 2013 17:36:13 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 54321 invoked by uid 500); 23 Apr 2013 17:36:13 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 54311 invoked by uid 99); 23 Apr 2013 17:36:13 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 23 Apr 2013 17:36:13 +0000 X-ASF-Spam-Status: No, hits=-2.3 required=5.0 tests=RCVD_IN_DNSWL_MED,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: local policy) Received: from [192.174.58.133] (HELO XEDGEB.nrel.gov) (192.174.58.133) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 23 Apr 2013 17:36:09 +0000 Received: from XHUBA.nrel.gov (10.20.4.58) by XEDGEB.nrel.gov (192.174.58.133) with Microsoft SMTP Server (TLS) id 8.3.298.1; Tue, 23 Apr 2013 11:35:45 -0600 Received: from MAILBOX2.nrel.gov ([fe80::19a0:6c19:6421:12f]) by XHUBA.nrel.gov ([::1]) with mapi; Tue, 23 Apr 2013 11:35:48 -0600 From: "Hiller, Dean" To: "user@cassandra.apache.org" Date: Tue, 23 Apr 2013 11:35:45 -0600 Subject: Re: Prepared Statement - cache duration (CQL3 - Cassandra 1.2.4) Thread-Topic: Prepared Statement - cache duration (CQL3 - Cassandra 1.2.4) Thread-Index: Ac5ASP11nnmbL7P1S8eXUoT7kdyXfA== Message-ID: In-Reply-To: Accept-Language: en-US Content-Language: en-US X-MS-Has-Attach: X-MS-TNEF-Correlator: user-agent: Microsoft-MacOutlook/14.3.2.130206 acceptlanguage: en-US Content-Type: text/plain; charset="us-ascii" Content-Transfer-Encoding: quoted-printable MIME-Version: 1.0 X-Virus-Checked: Checked by ClamAV on apache.org Nice, Thanks, Dean From: Sylvain Lebresne > Reply-To: "user@cassandra.apache.org" > Date: Tuesday, April 23, 2013 11:31 AM To: "user@cassandra.apache.org" > Subject: Re: Prepared Statement - cache duration (CQL3 - Cassandra 1.2.4) On Tue, Apr 23, 2013 at 6:02 PM, Hiller, Dean > wrote: Out of curiosity, why did cassandra choose to re-invent the wheel instead o= f using something like google protobuf which spans multiple languages? I see it as a step better than thrift since it is really only defining mess= age format and has all sorts of goodies with it. I think you only need to = frame it and that may exist already as well actually but I can't remember. The serialization/deserialization involved in the binary protocol is not a = big deal tbh, so I guess we chose to avoid the dependency. I personally don= 't think using protobufs would have simplified things much in practice, I d= on't think there is that much wheel reinventing, and so I'm reasonably happ= y to have something tailored to your needs. I'll admit there is some subjec= tivity in that opinion however, your mileage may vary. Lastly, does the java-driver have an asynch nature to it at all? The java driver is completely asynchronous, from the protocol to it's imple= mentation, so yes. It would be nice to be able to call driver.put(myData, myCallbackSuccessHa= ndler); In case you look at the driver API, its execute method returns a future, th= at happens to extends guava's ListenableFuture, and so you can add a callba= ck/listener through that. -- Sylvain From: Sylvain Lebresne >> Reply-To: "user@cassandra.apache.org>" >> Date: Tuesday, April 23, 2013 9:55 AM To: "user@cassandra.apache.org>" >> Subject: Re: Prepared Statement - cache duration (CQL3 - Cassandra 1.2.4) When we speak of "binary protocol", we talk about the protocol introduced i= n Cassandra 1.2 that is an alternative to thrift for CQL3. It's a custom, b= inary, protocol, that has not link to thrift whatsoever. That protocol is defined by the document here: https://git-wip-us.apache.or= g/repos/asf?p=3Dcassandra.git;a=3Dblob_plain;f=3Ddoc/native_protocol_v1.spe= c;hb=3DHEAD Of course, this is just a protocol, and unless you have the time and willin= gness to write a proper library using that protocol, you should just use an= existing driver implementing it. If you are using Java (some of your examp= le above seems to be in Java), then you could for instance pick https://git= hub.com/datastax/java-driver. If you're not using java, then well, since sa= id protocol is fairly recent, there isn't an existing driver for every lang= uages, but a bunch of drivers are in the work. That being said, I'm not saying you *should* use a driver that uses the bin= ary protocol, just that at least for exceptions handling, said binary proto= col has a slightly cleaner handling of them than what's available through t= hrift. I'll not that even if you do want to use thrift, it's usually advise= d to use a high level client rather than raw thrift. Unless you have no cho= ice or like suffering that is. -- Sylvain On Tue, Apr 23, 2013 at 5:38 PM, Stuart Broad >= > wrote: Hi Edward, My understanding was that thrift supports a number of protocols (binary bei= ng one of them). I don't understand what switching to "binary protocol" bu= t not using thrift means. Can you point me to any code examples? Regards, Stuart On Tue, Apr 23, 2013 at 4:21 PM, Edward Capriolo >> wrote: Having to catch the exception and parse it is a bit ugly, however this is c= lose to what someone might do with an SQLException to determine if the erro= r was transient etc. If there is an error code it is possible that it coul= d be added as an optional property of the InvalidRequestException in future= versions. Switching to the "binany protocol" is not a method in thrift, it means your= not using thrift at all. On Tue, Apr 23, 2013 at 11:13 AM, Stuart Broad = >> wrote: Hi Edward, Thanks for your reply - I was already using the prepare/execute cql methods= that you suggested. My problem is that these methods 'mask' the PreparedQ= ueryNotFoundException as an InvalidRequestException. At present I catch th= e InvalidRequestException (when cassandra has been re-started) and check th= e message text to figure out if I need to rebuild the prepared queries (rat= her than building each time I call). Sylvain had suggested that I use the binary protocol as the exceptions are = more explicit so I am trying to determine how this can be done (I don't see= any obvious methods other than the cql ones for calling prepared statement= s). Regards, Stuart On Tue, Apr 23, 2013 at 4:05 PM, Edward Capriolo >> wrote: Thrift has a prepare_cql call which returns an ID. Then it has an exececute= _cql call which takes the id and a map or variable bindings. On Tue, Apr 23, 2013 at 10:29 AM, Stuart Broad = >> wrote: Hi all, I just realised that the binary protocol is the low-level thrift api that I= was originally using (Cassandra.Client>> get / insert ...). How can a pre= pared statement be called through the thrift api (i.e. not the cql methods)= ? Cheers, Stuart On Tue, Apr 23, 2013 at 11:48 AM, Stuart Broad = >> wrote: Hi Sylvain, Thanks for your response. I am handling the 'PreparedQueryNotFoundExceptio= n' more for the case of a cassandra re-start (rather then expecting to buil= d 100000 statements). I am not familiar with the binary protocol - which class/methods should I l= ook at? Regards, Stuart On Tue, Apr 23, 2013 at 11:29 AM, Sylvain Lebresne >> wrote: In thrift, a lot of exceptions (like PreparedQueryNotFoundException) are si= mply returned as InvalidRequestException. The reason for that was a mix of = not wanting to change the thrift API too much and because we didn't knew ho= w to return a lot of different exception with thrift without making it horr= ible to work with. So you'll probably have to parse strings here indeed. This will be cleaner/less fragile if you use the binary protocol as excepti= ons are more fined grained there. Though taking a step back (and without saying that you shouldn't handle the= case where a query is not prepared on the node you contact), if you're rea= lly considering preparing more than 100000 statements, I'd suggest that it = might be worth benchmarking whether using prepared statements in your case = is really going to be worth the trouble. Just saying. -- Sylvain On Tue, Apr 23, 2013 at 12:14 PM, Stuart Broad = >> wrote: Hi Sorin, The PreparedQueryNotFoundException is not thrown from Cassandra.Client>>exe= cute_prepared_cql3_query method. I created some prepared statements and th= en re-started cassandra and received the following exception: InvalidRequestException(why: Prepared query with ID 1124421588 not found (e= ither the query was not prepared on this host (maybe the host has been rest= arted?) or you have prepared more than 100000 queries and queries 112442158= 8 has been evicted from the internal cache)) The best I have been able to come up with is the following: try { client.execute_prepared_cql3_query(psId, bindValues, ..); } catch (InvalidRequestException invEx) { String why =3D invEx.getWhy(); CLogger.logger().warning(why); if(why.startsWith("Prepared query with ID")) { rebuildPreparedStatement(preparedStatement); client.execute_prepared_cql3_query(psId, bindValues, ..= ); } else { throw invEx; } } Obviously this is pretty fragile and would break if the cassandra message w= as changed...but it least it works for now! Cheers, Stuart On Sun, Apr 21, 2013 at 11:51 AM, Sorin Manolache >> wrote: On 2013-04-19 13:57, Stuart Broad wrote: Hi, I am using Cassandra.Client prepare_cql3_query/execute_prepared_cql3_query to create and run some prepared statements. It is working well but I am unclear as to how long the server side 'caches' the prepared statements. Should a prepared statement be prepared for every new Cassandra.Client? Based on my limited testing it seems like I can create some prepared statements in one Cassandra.Client and use in another but I am not sure how reliable/lasting this is i.e. If I called the prepared statement again the next day would it still exist? What about if cassandra was re-started? _Background:_ I am creating prepared statements for batch updates of pre-defined lengths (e.g. 10000, 1000, 500, 250, 50, 10, 1) and wanted to know if these could just be set up once. We felt that using the prepared statements was easier than escaping values within a CQL statement and probably more performant. Thanks in advance for your help. I've looked in Cassandra's code (v1.2.3). The cache of prepared statements = has a size of 100,000. So if you prepare more than 100 thousand statements,= the least recently used ones will vanish. You'll get the exception Prepare= dQueryNotFoundException, code 0x2500. Regards, Sorin