cassandra tests

2012-10-10 Thread Alexey Zotov
Hi Guys,

I tried to write some tests for HH mechanism, but I couldn't do what I
want. Tests writing mechanism is inconvenient and should be improved. I
understand that it's not easy to fix it.

As the first step I suggest to include some mock framework, e.g. PowerMock
(as most powerful as I know), to the project.

Are there some another tests, except included to the project? If no, then I
suggest to create pack of integration tests based on ccm. It requires
additional writing of scripts for more deep configuration and management of
ccm's cluster.

What do you think about my thoughts?


Re: cassandra tests

2012-10-10 Thread Brandon Williams
On Wed, Oct 10, 2012 at 3:54 AM, Alexey Zotov  wrote:
> Are there some another tests, except included to the project? If no, then I
> suggest to create pack of integration tests based on ccm. It requires
> additional writing of scripts for more deep configuration and management of
> ccm's cluster.
>
> What do you think about my thoughts?

That's a great idea: https://github.com/riptano/cassandra-dtest

There's a basic HH test in consistency_test.py

-Brandon


Re: [VOTE] Release Apache Cassandra 1.1.6

2012-10-10 Thread Jonathan Ellis
Also: https://issues.apache.org/jira/browse/CASSANDRA-4782

On Tue, Oct 9, 2012 at 5:02 PM, Jonathan Ellis  wrote:
> Just committed a fix for possible infinite L0 compaction in LCS
> (f34bd79b9a92f23c1fc5e185e074d7faa880fc0b), can we include that?
>
> On Tue, Oct 9, 2012 at 11:12 AM, Jonathan Ellis  wrote:
>> +1
>>
>> On Tue, Oct 9, 2012 at 10:48 AM, Sylvain Lebresne  
>> wrote:
>>> Quite a few fixes since 1.1.5 and there's at least #4772 that makes it worth
>>> pushing a new release. So I propose the following artifacts for release as
>>> 1.1.6.
>>>
>>> sha1: 2773f7cd8ba5e9b3b293f26f546ddcd450831d82
>>> Git: 
>>> http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=shortlog;h=refs/tags/1.1.6-tentative
>>> Artifacts: 
>>> https://repository.apache.org/content/repositories/orgapachecassandra-038/org/apache/cassandra/apache-cassandra/1.1.6/
>>> Staging repository:
>>> https://repository.apache.org/content/repositories/orgapachecassandra-038/
>>>
>>> The artifacts as well as the debian package are also available here:
>>> http://people.apache.org/~slebresne/
>>>
>>> The vote will be open for 72 hours (longer if needed).
>>>
>>> [1]: http://goo.gl/MYRFT (CHANGES.txt)
>>> [2]: http://goo.gl/I9xPL (NEWS.txt)
>>
>>
>>
>> --
>> Jonathan Ellis
>> Project Chair, Apache Cassandra
>> co-founder of DataStax, the source for professional Cassandra support
>> http://www.datastax.com
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com


Possible issue with read repair?

2012-10-10 Thread Niklas Ekström
Hi,

I’m looking in the file StorageProxy.java (Cassandra 1.1.5), and line 766 seems 
odd to me.

FBUtilities.waitOnFutures() is called with the repairResults from the 
RowRepairResolver resolver.

The problem though is that repairResults is only assigned when the object is 
created at line 737 in StorageProxy.java, and there it is assigned to 
Collections.emptyList(), and in the resolve() method in RowRepairResolver, 
which is indirectly called from line 771 in StorageProxy.java, that is, after 
the call to FBUtilities.waitOnFutures().

So the effect is that line 766 in StorageProxy.java is essentially a no-op.

If on the other hand line 766 is moved down to just below the try-catch block 
under it (to line 777), the effect of the call to FBUtilities.waitOnFutures() 
would be to wait for responses to the READ_REPAIR message. Not waiting for 
responses to read repair messages opens a window of time in which stale reads 
can happen.

Does this sound reasonable or am I overlooking something?

Regards,
Niklas


Re: Possible issue with read repair?

2012-10-10 Thread Mikhail Panchenko
I'll take a stab:

Without looking at the code, that seems perfectly fine - the purpose of
read repair is to repair potentially stale data out of band. It is
acceptable (from the viewpoint of the datastore) to have "stale" reads
while read-repair happens in the background. Once the repair is completed,
future reads will have the correct data ("eventually"). Reads do not and
should not block on read repair tasks. See
http://www.datastax.com/docs/1.1/cluster_architecture/about_client_requests#about-read-requestsfor
more info.

In order to achieve what you're looking for and eliminate the window you
are describing, one would write and read at QUORUM consistency level.

On Wed, Oct 10, 2012 at 1:25 PM, Niklas Ekström  wrote:

> Hi,
>
> I’m looking in the file StorageProxy.java (Cassandra 1.1.5), and line 766
> seems odd to me.
>
> FBUtilities.waitOnFutures() is called with the repairResults from the
> RowRepairResolver resolver.
>
> The problem though is that repairResults is only assigned when the object
> is created at line 737 in StorageProxy.java, and there it is assigned to
> Collections.emptyList(), and in the resolve() method in RowRepairResolver,
> which is indirectly called from line 771 in StorageProxy.java, that is,
> after the call to FBUtilities.waitOnFutures().
>
> So the effect is that line 766 in StorageProxy.java is essentially a no-op.
>
> If on the other hand line 766 is moved down to just below the try-catch
> block under it (to line 777), the effect of the call to
> FBUtilities.waitOnFutures() would be to wait for responses to the
> READ_REPAIR message. Not waiting for responses to read repair messages
> opens a window of time in which stale reads can happen.
>
> Does this sound reasonable or am I overlooking something?
>
> Regards,
> Niklas
>


Possible issue with read repair?

2012-10-10 Thread Niklas Ekström
Hi,

I’m looking in the file StorageProxy.java (Cassandra 1.1.5), and line 766 seems 
odd to me.

FBUtilities.waitOnFutures() is called with the repairResults from the 
RowRepairResolver resolver.

The problem though is that repairResults is only assigned when the object is 
created at line 737 in StorageProxy.java, and there it is assigned to 
Collections.emptyList(), and in the resolve() method in RowRepairResolver, 
which is indirectly called from line 771 in StorageProxy.java, that is, after 
the call to FBUtilities.waitOnFutures().

So the effect is that line 766 in StorageProxy.java is essentially a no-op.

If on the other hand line 766 is moved down to just below the try-catch block 
under it (to line 777), the effect of the call to FBUtilities.waitOnFutures() 
would be to wait for responses to the READ_REPAIR message. Not waiting for 
responses to read repair messages opens a window of time in which stale reads 
can happen.

Does this sound reasonable or am I overlooking something?

Regards,
Niklas


Re: Possible issue with read repair?

2012-10-10 Thread Jonathan Ellis
You're both right -- "read repair" as a concept is indeed performed
asynchronously, but RowRepairResolver is used for synchronous, high-CL
reads as well, which is the code Niklas is referring to.

Niklas, can you create a ticket to fix this officially?

On Wed, Oct 10, 2012 at 3:31 PM, Mikhail Panchenko  wrote:
> I'll take a stab:
>
> Without looking at the code, that seems perfectly fine - the purpose of
> read repair is to repair potentially stale data out of band. It is
> acceptable (from the viewpoint of the datastore) to have "stale" reads
> while read-repair happens in the background. Once the repair is completed,
> future reads will have the correct data ("eventually"). Reads do not and
> should not block on read repair tasks. See
> http://www.datastax.com/docs/1.1/cluster_architecture/about_client_requests#about-read-requestsfor
> more info.
>
> In order to achieve what you're looking for and eliminate the window you
> are describing, one would write and read at QUORUM consistency level.
>
> On Wed, Oct 10, 2012 at 1:25 PM, Niklas Ekström  wrote:
>
>> Hi,
>>
>> I’m looking in the file StorageProxy.java (Cassandra 1.1.5), and line 766
>> seems odd to me.
>>
>> FBUtilities.waitOnFutures() is called with the repairResults from the
>> RowRepairResolver resolver.
>>
>> The problem though is that repairResults is only assigned when the object
>> is created at line 737 in StorageProxy.java, and there it is assigned to
>> Collections.emptyList(), and in the resolve() method in RowRepairResolver,
>> which is indirectly called from line 771 in StorageProxy.java, that is,
>> after the call to FBUtilities.waitOnFutures().
>>
>> So the effect is that line 766 in StorageProxy.java is essentially a no-op.
>>
>> If on the other hand line 766 is moved down to just below the try-catch
>> block under it (to line 777), the effect of the call to
>> FBUtilities.waitOnFutures() would be to wait for responses to the
>> READ_REPAIR message. Not waiting for responses to read repair messages
>> opens a window of time in which stale reads can happen.
>>
>> Does this sound reasonable or am I overlooking something?
>>
>> Regards,
>> Niklas
>>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com