incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stuart Broad <stu...@moogsoft.com>
Subject Re: Prepared Statement - cache duration (CQL3 - Cassandra 1.2.4)
Date Tue, 23 Apr 2013 16:01:31 GMT
Aha - got it.  Thanks for everyones help.

I think I will stick with the prepare/execute CQL (with the
InvalidRequestException check) for now.  I will take a look at the  driver
you mentioned though.

Cheers,

Stuart


On Tue, Apr 23, 2013 at 4:55 PM, Sylvain Lebresne <sylvain@datastax.com>wrote:

> When we speak of "binary protocol", we talk about the protocol introduced
> in Cassandra 1.2 that is an alternative to thrift for CQL3. It's a custom,
> binary, protocol, that has not link to thrift whatsoever.
>
> That protocol is defined by the document here:
> https://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=blob_plain;f=doc/native_protocol_v1.spec;hb=HEAD
>
> Of course, this is just a protocol, and unless you have the time and
> willingness 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
> example above seems to be in Java), then you could for instance pick
> https://github.com/datastax/java-driver. If you're not using java, then
> well, since said protocol is fairly recent, there isn't an existing driver
> for every languages, but a bunch of drivers are in the work.
>
> That being said, I'm not saying you *should* use a driver that uses the
> binary protocol, just that at least for exceptions handling, said binary
> protocol has a slightly cleaner handling of them than what's available
> through thrift. I'll not that even if you do want to use thrift, it's
> usually advised to use a high level client rather than raw thrift. Unless
> you have no choice or like suffering that is.
>
> --
> Sylvain
>
>
> On Tue, Apr 23, 2013 at 5:38 PM, Stuart Broad <stuart@moogsoft.com> wrote:
>
>> Hi Edward,
>>
>> My understanding was that thrift supports a number of protocols (binary
>> being one of them).  I don't understand what switching to "binary protocol"
>> but 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 <edlinuxguru@gmail.com>wrote:
>>
>>> Having to catch the exception and parse it is a bit ugly, however this
>>> is close to what someone might do with an SQLException to determine if the
>>> error was transient etc.  If there is an error code it is possible that it
>>> could 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 <stuart@moogsoft.com>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
>>>> PreparedQueryNotFoundException as an InvalidRequestException.  At present
I
>>>> catch the InvalidRequestException (when cassandra has been re-started) and
>>>> check the message text to figure out if I need to rebuild the prepared
>>>> queries (rather 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
>>>> statements).
>>>>
>>>> Regards,
>>>>
>>>> Stuart
>>>>
>>>>
>>>> On Tue, Apr 23, 2013 at 4:05 PM, Edward Capriolo <edlinuxguru@gmail.com
>>>> > 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 <stuart@moogsoft.com>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 prepared 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 <stuart@moogsoft.com>wrote:
>>>>>>
>>>>>>> Hi Sylvain,
>>>>>>>
>>>>>>> Thanks for your response.  I am handling the
>>>>>>> 'PreparedQueryNotFoundException' more for the case of a cassandra
re-start
>>>>>>> (rather then expecting to build 100000 statements).
>>>>>>>
>>>>>>> I am not familiar with the binary protocol - which class/methods
>>>>>>> should I look at?
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Stuart
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Apr 23, 2013 at 11:29 AM, Sylvain Lebresne <
>>>>>>> sylvain@datastax.com> wrote:
>>>>>>>
>>>>>>>> In thrift, a lot of exceptions (like
>>>>>>>> PreparedQueryNotFoundException) are simply 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
how to return a
>>>>>>>> lot of different exception with thrift without making it
horrible 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
>>>>>>>> exceptions 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 really 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 <stuart@moogsoft.com
>>>>>>>> > wrote:
>>>>>>>>
>>>>>>>>> Hi Sorin,
>>>>>>>>>
>>>>>>>>> The PreparedQueryNotFoundException is not thrown from
>>>>>>>>> Cassandra.Client>>execute_prepared_cql3_query method.
 I created some
>>>>>>>>> prepared statements and then re-started cassandra and
received the
>>>>>>>>> following exception:
>>>>>>>>>
>>>>>>>>> InvalidRequestException(why: Prepared query with ID 1124421588
not
>>>>>>>>> found (either the query was not prepared on this host
(maybe the host has
>>>>>>>>> been restarted?) or you have prepared more than 100000
queries and queries
>>>>>>>>> 1124421588 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 = 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 was changed...but it least it works for now!
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>>
>>>>>>>>> Stuart
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Sun, Apr 21, 2013 at 11:51 AM, Sorin Manolache <
>>>>>>>>> sorinm@gmail.com> 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 PreparedQueryNotFoundException**, code
0x2500.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Sorin
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Mime
View raw message