[jira] [Created] (CASSANDRA-9312) Provide a way to retrieve the write time of a CQL row

2015-05-05 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-9312:
--

 Summary: Provide a way to retrieve the write time of a CQL row
 Key: CASSANDRA-9312
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9312
 Project: Cassandra
  Issue Type: New Feature
  Components: API
Reporter: Nicolas Favre-Felix


There is currently no way to retrieve the "writetime" of a CQL row. This is an 
issue for tables in which all dimensions are part of the primary key.

Since Cassandra already stores a cell for the CQL row, it would make sense to 
provide a way to read its timestamp. This feature would be consistent with the 
concept of a row as an entity containing a number of optional columns, but able 
to exist on its own.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-6160) Throw errror when attempting to create a secondary index against counter

2015-01-28 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295654#comment-14295654
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6160:


Unfortunately this breaks secondary indexes on primary key dimensions:

{code}
cqlsh:ks1> create table test3 (col1 int, col2 int, col3 counter, primary 
key((col1, col2)));
cqlsh:ks1> update test3 set col3 = col3 + 3 where col1 = 1 and col2 = 2;
cqlsh:ks1> select * from test3;

 col1 | col2 | col3
--+--+--
1 |2 |3

(1 rows)

cqlsh:ks1> create index expected on test3(col2);
Bad Request: Secondary indexes are not supported on counter tables
{code}

With RMW counters, we should be able to add both the counter write and the 
secondary index update in the same commit log entry. I couldn't find a JIRA for 
this; is there one?

> Throw errror when attempting to create a secondary index against counter
> 
>
> Key: CASSANDRA-6160
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6160
> Project: Cassandra
>  Issue Type: Bug
>  Components: API
>Reporter: Adam Hattrell
>Assignee: Sylvain Lebresne
>Priority: Minor
> Fix For: 1.2.11
>
> Attachments: 6160.txt
>
>
> Using CQL you can create a secondary index against a counter which is then 
> non-functional.  
> {code}
> cqlsh:test> create table test2 (col1 int, col2 counter, primary key (col1)) ;
> cqlsh:test> create index dodgy on test2(col2) ;
> cqlsh:test> update test2 set col2 = col2 + 0 where col1 = 1 ;
> cqlsh:test> select * from test2 ;
>  col1 | col2
> --+--
> 1 |0
> cqlsh:t7088> select * from test2 where col2 = 0 ;
> {code}
> We should return an error to let users know they are in unsupported territory.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8254) Query parameters (and more) are limited to 65,536 entries

2014-11-04 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-8254:
--

 Summary: Query parameters (and more) are limited to 65,536 entries
 Key: CASSANDRA-8254
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8254
 Project: Cassandra
  Issue Type: Bug
  Components: API
Reporter: Nicolas Favre-Felix


Parameterized queries are sent over the wire as a string followed by a list of 
arguments. This list is decoded in QueryOptions.Codec by 
CBUtil.readValueList(body), which in turn reads a 16-bit short value from the 
wire as the number of values to deserialize.
Sending more values leads to a silent overflow, sometimes reported by the 
driver as a protocol error as other values are deserialized incorrectly.

64k sounds like a lot, but tables with a large number of clustering dimensions 
can hit this limit when fetching a few thousand CQL rows only with an IN query, 
e.g.

{code}
SELECT * FROM sensor_data WHERE a=? and (b,c,d,e,f,g,h,i) IN 
((?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?) ... 
)
{code}

Here, having 8 dimensions in the clustering key plus 1 in the partitioning key 
restricts the read to 8,191 CQL rows.

Some other parts of Cassandra still use 16-bit sizes, for example preventing 
users to fetch all elements of a large collection (CASSANDRA-6428). The 
suggestion at the time was "we'll fix it in the next iteration of the binary 
protocol", so I'd like to suggest switching to variable-length integers as this 
would solve such issues while keeping messages short.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8051) Add SERIAL and LOCAL_SERIAL consistency levels to cqlsh

2014-10-03 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-8051:
--

 Summary: Add SERIAL and LOCAL_SERIAL consistency levels to cqlsh
 Key: CASSANDRA-8051
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8051
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Reporter: Nicolas Favre-Felix
Priority: Minor


cqlsh does not support setting the serial consistency level. The default 
CL.SERIAL does not let users safely execute LWT alongside an app that runs at 
LOCAL_SERIAL, and can prevent any LWT from running when a DC is down (e.g. with 
2 DCs, RF=3 in each.)

Implementing this well is a bit tricky. A user setting the serial CL will 
probably not want all of their statements to have a serial CL attached, but 
only the conditional updates. At the same time it would be useful to support 
serial reads. "WITH CONSISTENCY LEVEL" used to provide this flexibility.

I believe that it is currently impossible to run a SELECT at SERIAL or 
LOCAL_SERIAL; the only workaround seems to be to run a conditional update with 
a predicate that always resolves to False, and to rely on the CAS response to 
read the data.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-7844) Fetching a single static column requires scanning to the first live CQL row

2014-08-27 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-7844:
--

 Summary: Fetching a single static column requires scanning to the 
first live CQL row
 Key: CASSANDRA-7844
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7844
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Nicolas Favre-Felix


Reading a single static column seems to do more work than needed, scanning the 
partition until the first live CQL row before returning a value.

As I understand, static columns are stored separately from clustered rows (see 
CASSANDRA-6956 for an issue that arised from this storage model). Nevertheless, 
Cassandra doesn't optimize for the case where only static columns and partition 
key dimensions are retrieved.

Selecting a static column on its own is possible:

{code}
> create table friends (user text, next_id int static, friend_id int, email 
> text, primary key(user,friend_id));
> insert into friends (user, next_id) values ('user1', 1);
> select * from friends where user = 'user1';

 user  | friend_id | next_id | email
---+---+-+---
 user1 |  null |   1 |  null

(1 rows)
{code}


Let's insert and delete some clustered data:

{code}
> insert into friends (user, next_id, friend_id, email) values ('user1', 2, 1, 
> 'abc@foo');
> insert into friends (user, next_id, friend_id, email) values ('user1', 3, 2, 
> 'def@foo');
> insert into friends (user, next_id, friend_id, email) values ('user1', 4, 3, 
> 'ghi@foo');
> select * from friends where user = 'user1';

 user  | friend_id | next_id | email
---+---+-+-
 user1 | 1 |   4 | abc@foo
 user1 | 2 |   4 | def@foo
 user1 | 3 |   4 | ghi@foo

(3 rows)


> delete from friends where user = 'user1' and friend_id = 1;
> delete from friends where user = 'user1' and friend_id = 2;
> delete from friends where user = 'user1' and friend_id = 3;
{code}


And then fetch the static column again:

{code}
> TRACING ON
Now tracing requests.
> select next_id from friends where user = 'user1' limit 1;

 next_id
-
   4

(1 rows)


Tracing session: 597cc970-2e27-11e4-932f-c551d8e65d14

 activity  | 
timestamp| source| source_elapsed
---+--+---+
execute_cql3_query | 
13:18:46,792 | 127.0.0.1 |  0
 Parsing SELECT next_id from friends where user = 'user1' LIMIT 1; | 
13:18:46,792 | 127.0.0.1 | 59
   Preparing statement | 
13:18:46,792 | 127.0.0.1 |125
   Executing single-partition query on friends | 
13:18:46,792 | 127.0.0.1 |357
  Acquiring sstable references | 
13:18:46,792 | 127.0.0.1 |369
   Merging memtable tombstones | 
13:18:46,792 | 127.0.0.1 |381
 Skipped 0/0 non-slice-intersecting sstables, included 0 due to tombstones | 
13:18:46,792 | 127.0.0.1 |445
Merging data from memtables and 0 sstables | 
13:18:46,792 | 127.0.0.1 |460
Read 1 live and 6 tombstoned cells | 
13:18:46,792 | 127.0.0.1 |504
  Request complete | 
13:18:46,792 | 127.0.0.1 |711
{code}


We went over tombstones instead of returning the static column immediately.

Is this possibly related to CASSANDRA-7085?



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7531) Node replacement leads to null entries in system.peers

2014-07-11 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14059128#comment-14059128
 ] 

Nicolas Favre-Felix commented on CASSANDRA-7531:


Brandon, it was a hand-rolled 2.0.9 with a few small changes, all unrelated to 
Gossip or cluster metadata.

> Node replacement leads to null entries in system.peers
> --
>
> Key: CASSANDRA-7531
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7531
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Nicolas Favre-Felix
>Assignee: Brandon Williams
>
> On 2.0.9, I'm still reproducing a similar behavior as in CASSANDRA-7122.
> One node was replaced and it's no longer in nodetool status but still has 
> null entries in system.peers:
> {code}
>  peer  | data_center | host_id  | 
> preferred_ip | rack| release_version | rpc_address   | schema_version 
>   | tokens
> ---+-+--+--+-+-+---+--+--
>   10.123.99.36 | DC1 | null | 
> null |null |null |  null |
>  null | null
>  10.123.45.121 | DC1 | e34b134c-c03e-464e-99a9-e3c022eed369 | 
> null | rack123 |  2.0.9-SNAPSHOT | 10.123.45.121 | 
> e6b32c6b-f3f3-3350-9bd1-a9b8620d42ea |  {'4099276460824344804'}
>   10.123.77.88 | DC2 | fdd76fa0-0971-43d5-961b-452bd2821b31 | 
> null | rack456 |  2.0.9-SNAPSHOT |  10.123.77.88 | 
> e6b32c6b-f3f3-3350-9bd1-a9b8620d42ea | {'-3074457345618258603'}
> [...]
> {code}
> This causes errors to be logged from the datastax Java driver (2.0.2):
> {code}
> [main] ERROR com.datastax.driver.core.ControlConnection - No rpc_address 
> found for host /10.123.99.36 in 
> my.anonymized.host.name.com/10.123.45.66:9042's peers system table. That 
> should not happen but using address /10.123.99.36 instead
> {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7537) Updates and partition tombstones are not given the same timestamp in a CAS batch

2014-07-11 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-7537:
--

 Summary: Updates and partition tombstones are not given the same 
timestamp in a CAS batch
 Key: CASSANDRA-7537
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7537
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Nicolas Favre-Felix


Create a table with one partition and 2 CQL rows:
{code}
CREATE TABLE t1 (
k text,
c text,
v text,
PRIMARY KEY(k,c)
);

BEGIN BATCH
INSERT INTO t1 (k,c,v) VALUES ('x','1','1');
INSERT INTO t1 (k,c,v) VALUES ('x','2','2');
APPLY BATCH;
{code}

CAS-delete the full partition based on the expected value of a single column:
{code}
cqlsh:ks1> SELECT * FROM t1 WHERE k='x';

 k | c | v
---+---+---
 x | 1 | 1
 x | 2 | 2

(2 rows)

cqlsh:ks1> BEGIN BATCH
   ... UPDATE t1 SET v = '0' WHERE k = 'x' AND c = '1' IF v = '1';
   ... DELETE FROM t1 WHERE k = 'x';
   ... APPLY BATCH;

 [applied]
---
  True

cqlsh:ks1> SELECT * FROM t1 WHERE k='x';

 k | c | v
---+---+---
 x | 1 | 0

(1 rows)
{code}

sstable2json reports that the updated column has a timestamp 1 greater than the 
partition delete:

{code}
{"key": "78","metadata": {"deletionInfo": 
{"markedForDeleteAt":1405097039224999,"localDeletionTime":1405097039}},"columns":
 [["1:v","0",1405097039225000]]}
{code}

All mutations in a CAS batch should be applied with the same timestamp.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7531) Node replacement leads to null entries in system.peers

2014-07-09 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-7531:
--

 Summary: Node replacement leads to null entries in system.peers
 Key: CASSANDRA-7531
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7531
 Project: Cassandra
  Issue Type: Bug
Reporter: Nicolas Favre-Felix


On 2.0.9, I'm still reproducing a similar behavior as in CASSANDRA-7122.
One node was replaced and it's no longer in nodetool status but still has null 
entries in system.peers:

{code}
 peer  | data_center | host_id  | 
preferred_ip | rack| release_version | rpc_address   | schema_version   
| tokens
---+-+--+--+-+-+---+--+--
  10.123.99.36 | DC1 | null | 
null |null |null |  null |  
   null | null
 10.123.45.121 | DC1 | e34b134c-c03e-464e-99a9-e3c022eed369 | 
null | rack123 |  2.0.9-SNAPSHOT | 10.123.45.121 | 
e6b32c6b-f3f3-3350-9bd1-a9b8620d42ea |  {'4099276460824344804'}
  10.123.77.88 | DC2 | fdd76fa0-0971-43d5-961b-452bd2821b31 | 
null | rack456 |  2.0.9-SNAPSHOT |  10.123.77.88 | 
e6b32c6b-f3f3-3350-9bd1-a9b8620d42ea | {'-3074457345618258603'}
[...]
{code}

This causes errors to be logged from the datastax Java driver (2.0.2):

{code}
[main] ERROR com.datastax.driver.core.ControlConnection - No rpc_address found 
for host /10.123.99.36 in my.anonymized.host.name.com/10.123.45.66:9042's peers 
system table. That should not happen but using address /10.123.99.36 instead
{code}




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7081) select writetime(colname) returns 0 for static columns

2014-04-23 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-7081:
--

 Summary: select writetime(colname) returns 0 for static columns
 Key: CASSANDRA-7081
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7081
 Project: Cassandra
  Issue Type: Bug
Reporter: Nicolas Favre-Felix


Selecting the write time for a static column returns 0 in Cassandra 2.0 
(c3550fe) and an expected timestamp in 2.1 (trunk, acdbbb9). Would it be 
possible to include this timestamp in a 2.0 release too?

{code}
> CREATE TABLE test (partition_key text, cluster_key text, data text, st text 
> static, PRIMARY KEY(partition_key, cluster_key));
> INSERT INTO test (partition_key, cluster_key, data, st) VALUES ( 'PK', 'CK', 
> 'DATA', 'ST');
> SELECT writetime(st), writetime(data) FROM test where partition_key='PK';

 writetime(st) | writetime(data)
---+--
 0 | 1398314681729000

(1 rows)
{code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6926) StorageService exposes two different objects for probability tracing

2014-03-25 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6926:
--

 Summary: StorageService exposes two different objects for 
probability tracing
 Key: CASSANDRA-6926
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6926
 Project: Cassandra
  Issue Type: Bug
Reporter: Nicolas Favre-Felix
Priority: Minor


StorageServiceMBean exposes the tracing probability using two different methods:

{code}
public void setTraceProbability(double probability);
public double getTracingProbability();
{code}

In a JMX explorer like JConsole, two objects are presented: one called 
TraceProbability and the other called TracingProbability. One is read-only, the 
other write-only; this is a bit confusing.

It is possible to have a single object there by having the same suffix for both 
the getter and setter.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6875) CQL3: select multiple CQL rows in a single partition using IN

2014-03-17 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6875:
--

 Summary: CQL3: select multiple CQL rows in a single partition 
using IN
 Key: CASSANDRA-6875
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6875
 Project: Cassandra
  Issue Type: Bug
  Components: API
Reporter: Nicolas Favre-Felix


In the spirit of CASSANDRA-4851 and to bring CQL to parity with Thrift, it is 
important to support reading several distinct CQL rows from a given partition 
using a distinct set of "coordinates" for these rows within the partition.

CASSANDRA-4851 introduced a range scan over the multi-dimensional space of 
clustering keys. We also need to support a "multi-get" of CQL rows, potentially 
using the "IN" keyword to define a set of clustering keys to fetch at once.

(reusing the same example\:)

Consider the following table:
{code}
CREATE TABLE test (
  k int,
  c1 int,
  c2 int,
  PRIMARY KEY (k, c1, c2)
);
{code}
with the following data:
{code}
 k | c1 | c2
---++
 0 |  0 |  0
 0 |  0 |  1
 0 |  1 |  0
 0 |  1 |  1
{code}

We can fetch a single row or a range of rows, but not a set of them:

{code}
> SELECT * FROM test WHERE k = 0 AND (c1, c2) IN ((0, 0), (1,1)) ;
Bad Request: line 1:54 missing EOF at ','
{code}

Supporting this syntax would return:
{code}
 k | c1 | c2
---++
 0 |  0 |  0
 0 |  1 |  1
{code}

Being able to fetch these two CQL rows in a single read is important to 
maintain partition-level isolation.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6837) Batch CAS does not support LOCAL_SERIAL

2014-03-11 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6837:
--

 Summary: Batch CAS does not support LOCAL_SERIAL
 Key: CASSANDRA-6837
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6837
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Nicolas Favre-Felix


The batch CAS feature introduced in Cassandra 2.0.6 does not support the 
LOCAL_SERIAL consistency level, and always uses SERIAL.

Create a cluster with 4 nodes with the following topology:

{code}
Datacenter: DC2
===
Status=Up/Down
|/ State=Normal/Leaving/Joining/Moving
--  AddressLoad   Tokens  Owns   Host ID   
Rack
UN  127.0.0.3  269 KB 256 26.3%  ae92d997-6042-42d9-b447-943080569742  
RAC1
UN  127.0.0.4  197.81 KB  256 25.1%  3edc92d7-9d1b-472a-8452-24dddbc4502c  
RAC1
Datacenter: DC1
===
Status=Up/Down
|/ State=Normal/Leaving/Joining/Moving
--  AddressLoad   Tokens  Owns   Host ID   
Rack
UN  127.0.0.1  226.92 KB  256 24.8%  dbc17bd7-1ede-47a2-9b31-6063752d6eb3  
RAC1
UN  127.0.0.2  179.27 KB  256 23.7%  bb0ad285-34d2-4989-a664-b068986ab6fa  
RAC1
{code}

In cqlsh:
{code}
cqlsh> CREATE KEYSPACE foo WITH replication = {'class': 
'NetworkTopologyStrategy', 'DC1': 2, 'DC2': 2};
cqlsh> USE foo;
cqlsh:foo> CREATE TABLE bar (x text, y bigint, z bigint, t bigint, PRIMARY 
KEY(x,y));
{code}

Kill nodes 127.0.0.3 and 127.0.0.4:

{code}
Datacenter: DC2
===
Status=Up/Down
|/ State=Normal/Leaving/Joining/Moving
--  AddressLoad   Tokens  Owns   Host ID   
Rack
DN  127.0.0.3  262.37 KB  256 26.3%  ae92d997-6042-42d9-b447-943080569742  
RAC1
DN  127.0.0.4  208.04 KB  256 25.1%  3edc92d7-9d1b-472a-8452-24dddbc4502c  
RAC1
Datacenter: DC1
===
Status=Up/Down
|/ State=Normal/Leaving/Joining/Moving
--  AddressLoad   Tokens  Owns   Host ID   
Rack
UN  127.0.0.1  214.82 KB  256 24.8%  dbc17bd7-1ede-47a2-9b31-6063752d6eb3  
RAC1
UN  127.0.0.2  178.23 KB  256 23.7%  bb0ad285-34d2-4989-a664-b068986ab6fa  
RAC1
{code}

Connect to 127.0.0.1 in DC1 and run a CAS batch at CL.LOCAL_SERIAL+LOCAL_QUORUM:

{code}
final Cluster cluster = new Cluster.Builder()
.addContactPoint("127.0.0.1")
.withLoadBalancingPolicy(new DCAwareRoundRobinPolicy("DC1"))
.build();

final Session session = cluster.connect("foo");

Batch batch = QueryBuilder.batch();
batch.add(new SimpleStatement("INSERT INTO bar (x,y,z) VALUES ('abc', 
123, 1) IF NOT EXISTS"));
batch.add(new SimpleStatement("UPDATE bar SET t=2 WHERE x='abc' AND 
y=123"));

batch.setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM);
batch.setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL);

session.execute(batch);
{code}

The batch fails with:

{code}
Caused by: com.datastax.driver.core.exceptions.UnavailableException: Not enough 
replica available for query at consistency SERIAL (3 required but only 2 alive)
at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:44)
at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:33)
at 
com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:182)
at 
org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66)
... 21 more
{code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6836) WriteTimeoutException always reports that the serial CL is "SERIAL"

2014-03-11 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6836:
--

 Summary: WriteTimeoutException always reports that the serial CL 
is "SERIAL"
 Key: CASSANDRA-6836
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6836
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Nicolas Favre-Felix
Priority: Minor


In StorageProxy.proposePaxos, the WriteTimeoutException is thrown with 
information about the consistency level. This CL is hardcoded to 
ConsistencyLevel.SERIAL, which might be wrong when LOCAL_SERIAL is used:

{code}
if (timeoutIfPartial && !callback.isFullyRefused())
throw new WriteTimeoutException(WriteType.CAS, 
ConsistencyLevel.SERIAL, callback.getAcceptCount(), requiredParticipants);
{code}

Suggested fix: pass consistencyForPaxos as a parameter to proposePaxos().



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6820) NPE in MeteredFlusher.run

2014-03-07 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6820:
--

 Summary: NPE in MeteredFlusher.run
 Key: CASSANDRA-6820
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6820
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Nicolas Favre-Felix
Priority: Minor


Hello,

I've been seeing this exception with Cassandra 2.0.5:

{code}
ERROR 15:41:46,754 Exception in thread Thread[OptionalTasks:1,5,main]
java.lang.NullPointerException
at org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java:40)
at 
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:75)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
{code}

Could it be that {{Memtable.activelyMeasuring}} becomes null right after the 
test?



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-18 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13904336#comment-13904336
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


Ah, my apologies for the confusion, I didn't realize the null was due to the 2i 
issue.
Selecting a single CQL row does include the static columns now.

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's comments for performance 
> reason.  So that "fetch a post and it's 50 last comments" is just one slice 
> internally.
> # you want to CAS rows of a dynamic partition based on some partition 
> condition. This is the same use case than why CASSANDRA-5633 exists for.
> As said above, 1) is already covered by collections, but 2) and 3) are not 
> (and
> I strongly believe collections are not the right fit, API wise, for those).
> Also, note that I don't want to underestimate the usefulness of 2). In most 
> cases, using a separate table for the blog posts and the comments is The 
> Right Solution, and trying to do 2) is premature optimisation. Yet, when used 
> properly, that kind of optimisation can make a difference, so I think having 
> a relatively native solution for it in CQL3 could make sense.
> Regarding 3), though CASSANDRA-5633 would provide one solution for it, I have 
> the feeling that static columns actually are a more natural approach (in term 
> of API). That's arguably more of a personal opinion/feeling though.
> So long story short, CQL3 lacks a way to mix both some "static" and "dynamic" 
> rows in the same partition of the same CQL3 table, and I think such a tool 
> could have it's use.
> The proposal is thus to allow "static" columns. Static columns would only 
> make sense in table with clustering columns (the "dynamic" ones). A static 
> column value would be static to the partition (all rows of the partition 
> would share the value for such column). The syntax would just be:
> {noformat}
> CREATE TABLE t (
>   k text,
>   s text static,
>   i int,
>   v text,
>   PRIMARY KEY (k, i)
> )
> {noformat}
> then you'd get:
> {noformat}
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm shared",   0, "foo");
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm still shared", 1, "bar");
> SELECT * FROM t;
>  k |  s | i |v
> 
> k0 | "I'm still shared" | 0 | "bar"
> k0 | "I'm still shared" | 1 | "foo"
> {noformat}
> There would be a few semantic details to decide on regarding deletions, ttl, 
> etc. but let's see if we agree it's a good idea first before ironing those 
> out.
> One last point is the implementation. Though I do think this idea has merits, 
> it's definitively not useful enough to justify rewriting the storage engine 
> for it. But I think we can support this relatively easily (emphasis on 
> "relatively" :)), which is probably the main reason why I like the approach.
> Namely, internally, we can store static columns as cells whose clustering 
> column values are empty. So in terms of cells, the partition of my example 
> would look like:
> {noformat}
> "k0" : [
>   (:"s" -> "I'm still shared"), // the static column
>   (0:"" -> "")  // row marker
>   (0:"v" -> "bar")
>   (1:"" -> "")  

[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-17 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13903785#comment-13903785
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


[~slebresne], my point was not about 2i, but about the fact that 
partition-level isolation is only useful if during _read_ you can select both 
the static and the clustered columns. An isolation property is used to 
guarantee that readers do not see partially-applied updates. If readers have to 
issue two queries to select the static and the clusters columns, they cannot 
rely on any isolation. They _will_ see partially applied updates and have an 
inconsistent view of the partition.

There is currently no way to select together both the static and clustered 
columns of a CQL row, even though this is how they are returned when the full 
partition is queried.

There are two more points on this subject:
# It seems to me that presenting CQL rows with the static columns for 
full-partition scans but not doing so in any other case displays a level of 
inconsistency in the API.
# All of this is pretty easy to do with Thrift.

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's comments for performance 
> reason.  So that "fetch a post and it's 50 last comments" is just one slice 
> internally.
> # you want to CAS rows of a dynamic partition based on some partition 
> condition. This is the same use case than why CASSANDRA-5633 exists for.
> As said above, 1) is already covered by collections, but 2) and 3) are not 
> (and
> I strongly believe collections are not the right fit, API wise, for those).
> Also, note that I don't want to underestimate the usefulness of 2). In most 
> cases, using a separate table for the blog posts and the comments is The 
> Right Solution, and trying to do 2) is premature optimisation. Yet, when used 
> properly, that kind of optimisation can make a difference, so I think having 
> a relatively native solution for it in CQL3 could make sense.
> Regarding 3), though CASSANDRA-5633 would provide one solution for it, I have 
> the feeling that static columns actually are a more natural approach (in term 
> of API). That's arguably more of a personal opinion/feeling though.
> So long story short, CQL3 lacks a way to mix both some "static" and "dynamic" 
> rows in the same partition of the same CQL3 table, and I think such a tool 
> could have it's use.
> The proposal is thus to allow "static" columns. Static columns would only 
> make sense in table with clustering columns (the "dynamic" ones). A static 
> column value would be static to the partition (all rows of the partition 
> would share the value for such column). The syntax would just be:
> {noformat}
> CREATE TABLE t (
>   k text,
>   s text static,
>   i int,
>   v text,
>   PRIMARY KEY (k, i)
> )
> {noformat}
> then you'd get:
> {noformat}
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm shared",   0, "foo");
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm still shared", 1, "bar");
> SELECT * FROM t;
>  k |  s | i |v
> 
> k0 | "I'm still shared" | 0 | "bar"
> k0 | "I'm still shared" | 1 | "foo"
> {noformat}
> There would be a few semantic details to decid

[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-17 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13903542#comment-13903542
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


Thanks Sylvain for the new patches, it looks great.

bq. This query should throw IRE, just like DELETEs do now.

I think this is a problem since there is no way to retrieve both a CQL row and 
the partition's static columns in a single SELECT.
This is an issue since partition-level isolation guarantees that you don't see 
partial updates within a partition; if the whole point of static columns is to 
have a consistent view of both clustered and unclustered data within a 
partition, not being able to fetch both in a single operation makes this 
isolation property useless.

Keeping the example of bills that have or haven't been paid, here's a table 
definition:
{code}
CREATE TABLE bills (
user text,
balance bigint  static,
expense_id bigint,
amount bigint,
item text,
paid boolean,
PRIMARY KEY (user, expense_id)
);

CREATE INDEX unpaid ON bills (paid);
{code}

Let's create 2 expenses for a single user, with CAS updates:
{code}

BEGIN BATCH
INSERT INTO bills (user, expense_id, amount, item, paid) values ('user1', 
1000, 8, 'burrito', false);
INSERT INTO bills (user, balance) VALUES ('user1', -8) IF NOT EXISTS;
APPLY BATCH;


BEGIN BATCH
INSERT INTO bills (user, expense_id, amount, item, paid) values ('user1', 
2000, 200, 'hotel room', false);
UPDATE bills SET balance = -208 WHERE user='user1' IF balance = -8;
APPLY BATCH;
{code}

They are both present:
{code}
> SELECT * FROM bills WHERE user='user1';

 user  | expense_id | balance | amount | item   | paid
---++-+++---
 user1 |   1000 |-208 |  8 |burrito | False
 user1 |   2000 |-208 |200 | hotel room | False

(2 rows)
{code}

The great thing about using a single partition that's updated with CAS is that 
all queries that read the full partition will always see a consistent view of 
the data, and respect our invariants – in our case, that the sum of the amount 
for all unpaid bills + the balance is equal to zero.

We can pay bills using CAS too – let's pay for the burrito:

{code}
BEGIN BATCH
UPDATE bills SET paid=true WHERE user='user1' AND expense_id=1000;
UPDATE bills SET balance=-200 WHERE user='user1' IF balance=-208;
APPLY BATCH;
{code}

This works, of course, and any client that retrieve the full partition would 
either see balance=-208 and all bills unpaid, or balance=-200 and one bill 
paid, but never anything else.

If we don't return the balance with the bill in a single SELECT, we lose the 
isolation property and the query for the balance could be out of date with the 
query for the bills themselves (SELECT * FROM bills WHERE user='user1' AND 
paid=false\;)

I'd argue that it is also confusing for users to see CQL rows with static 
columns filled in when they select the full partition but can't access the same 
data if they give the full PK coordinates of that row: you'd expect the second 
query to select a subset of the data extracted by the first.

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's co

[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-14 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13901408#comment-13901408
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


Hi,

I'd like to add that it is important to be able to fetch both a static column 
and clustered columns in a single select; this does not seem possible at the 
moment:

{code}
cqlsh:ks1> CREATE TABLE foo (
  x text,
  y bigint,
  t bigint static,
  z bigint,
  PRIMARY KEY (x, y)
);

cqlsh:ks1> insert into foo (x,y,z) values ('a', 1, 10);
cqlsh:ks1> insert into foo (x,y,z) values ('a', 2, 20);
cqlsh:ks1> update foo set t = 2 where x='a';
cqlsh:ks1> select * from foo;

 x | y | t | z
---+---+---+
 a | 1 | 2 | 10
 a | 2 | 2 | 20

(2 rows)
{code}

Here we have a select over a whole partition and it pulls the static column 
just fine. Selecting a CQL row works, of course, and selecting a static column 
does too:

{code}
cqlsh:ks1> select x,y,z from foo where x='a' and y=1;

 x | y | z
---+---+
 a | 1 | 10

(1 rows)

cqlsh:ks1> select t from foo where x='a';

 t
---
 2

(1 rows)
{code}

But selecting them together fails to return anything:
{code}
cqlsh:ks1> select x,y,z,t from foo where x='a' and y=1;

(0 rows)
{code}

Now this does partly make sense because there isn't really a value for "t" 
where y=1 since "t" isn't clustered. But it is important to be consistent with 
the output for the full table.

Note that querying the full partition returns only the static column now:

{code}
cqlsh:ks1> select x,y,z,t from foo where x='a';

 x | y| z| t
---+--+--+---
 a | null | null | 4

(1 rows)
{code}

Currently, the patches add support for:
* Selecting a CQL row by primary key (that's a standard feature).
* Selecting a static column by partition key (added by Sylvain).

So I'd say it's important to be able to support:
* Selecting clustered as well as static columns for a given CQL row.
* Selecting clustered as well as static columns for a given partition.

Not being able to fetch both the static column and a CQL row or set of CQL rows 
in a single read makes it impossible to rely on partition-level isolation for 
consistent reads.

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's comments for performance 
> reason.  So that "fetch a post and it's 50 last comments" is just one slice 
> internally.
> # you want to CAS rows of a dynamic partition based on some partition 
> condition. This is the same use case than why CASSANDRA-5633 exists for.
> As said above, 1) is already covered by collections, but 2) and 3) are not 
> (and
> I strongly believe collections are not the right fit, API wise, for those).
> Also, note that I don't want to underestimate the usefulness of 2). In most 
> cases, using a separate table for the blog posts and the comments is The 
> Right Solution, and trying to do 2) is premature optimisation. Yet, when used 
> properly, that kind of optimisation can make a difference, so I think having 
> a relatively native solution for it in CQL3 could make sense.
> Regarding 3), though CASSANDRA-5633 would provide one solution for it, I have 
> the feeling that static columns actually are a more natural approach (in term 
> of API). That's arguably more of a per

[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-06 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13893221#comment-13893221
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


[~iamaleksey]: the patch above enables CAS on several cells within a single 
partition too, in addition to supporting CAS on a static column.
For example:

{code}
BEGIN BATCH
UPDATE transactions SET paid='Y' WHERE user='foo' AND txid='123' IF 
tx_token='abc';
UPDATE transactions SET paid='Y' WHERE user='foo' AND txid='456' IF 
tx_token='def';
UPDATE transactions SET paid='Y' WHERE user='foo' AND txid='789' IF 
tx_token='ghi';
APPLY BATCH;
{code}

This is already something we can do with Thrift and composite columns; it is 
very useful to CAS-update several CQL rows within the same partition.

Sylvain's branch introduces support for both multi-CQL-row CAS in a partition 
with any number of cells to check as well as support for a static column within 
a partition. Both are important, imho.

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's comments for performance 
> reason.  So that "fetch a post and it's 50 last comments" is just one slice 
> internally.
> # you want to CAS rows of a dynamic partition based on some partition 
> condition. This is the same use case than why CASSANDRA-5633 exists for.
> As said above, 1) is already covered by collections, but 2) and 3) are not 
> (and
> I strongly believe collections are not the right fit, API wise, for those).
> Also, note that I don't want to underestimate the usefulness of 2). In most 
> cases, using a separate table for the blog posts and the comments is The 
> Right Solution, and trying to do 2) is premature optimisation. Yet, when used 
> properly, that kind of optimisation can make a difference, so I think having 
> a relatively native solution for it in CQL3 could make sense.
> Regarding 3), though CASSANDRA-5633 would provide one solution for it, I have 
> the feeling that static columns actually are a more natural approach (in term 
> of API). That's arguably more of a personal opinion/feeling though.
> So long story short, CQL3 lacks a way to mix both some "static" and "dynamic" 
> rows in the same partition of the same CQL3 table, and I think such a tool 
> could have it's use.
> The proposal is thus to allow "static" columns. Static columns would only 
> make sense in table with clustering columns (the "dynamic" ones). A static 
> column value would be static to the partition (all rows of the partition 
> would share the value for such column). The syntax would just be:
> {noformat}
> CREATE TABLE t (
>   k text,
>   s text static,
>   i int,
>   v text,
>   PRIMARY KEY (k, i)
> )
> {noformat}
> then you'd get:
> {noformat}
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm shared",   0, "foo");
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm still shared", 1, "bar");
> SELECT * FROM t;
>  k |  s | i |v
> 
> k0 | "I'm still shared" | 0 | "bar"
> k0 | "I'm still shared" | 1 | "foo"
> {noformat}
> There would be a few semantic details to decide on regarding deletions, ttl, 
> etc. but let's see if we agree it's a good idea first before ironing those 
> out.
> One 

[jira] [Commented] (CASSANDRA-6561) Static columns in CQL3

2014-02-05 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13892512#comment-13892512
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6561:


Thanks Sylvain. One more thing: it seems that the "static" suffix is not 
currently added to the column definition printed by "DESCRIBE TABLE foo;"

> Static columns in CQL3
> --
>
> Key: CASSANDRA-6561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6561
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0.6
>
>
> I'd like to suggest the following idea for adding "static" columns to CQL3.  
> I'll note that the basic idea has been suggested by jhalliday on irc but the 
> rest of the details are mine and I should be blamed for anything stupid in 
> what follows.
> Let me start with a rational: there is 2 main family of CF that have been 
> historically used in Thrift: static ones and dynamic ones. CQL3 handles both 
> family through the presence or not of clustering columns. There is however 
> some cases where mixing both behavior has its use. I like to think of those 
> use cases as 3 broad category:
> # to denormalize small amounts of not-entirely-static data in otherwise 
> static entities. It's say "tags" for a product or "custom properties" in a 
> user profile. This is why we've added CQL3 collections. Importantly, this is 
> the *only* use case for which collections are meant (which doesn't diminishes 
> their usefulness imo, and I wouldn't disagree that we've maybe not 
> communicated this too well).
> # to optimize fetching both a static entity and related dynamic ones. Say you 
> have blog posts, and each post has associated comments (chronologically 
> ordered). *And* say that a very common query is "fetch a post and its 50 last 
> comments". In that case, it *might* be beneficial to store a blog post 
> (static entity) in the same underlying CF than it's comments for performance 
> reason.  So that "fetch a post and it's 50 last comments" is just one slice 
> internally.
> # you want to CAS rows of a dynamic partition based on some partition 
> condition. This is the same use case than why CASSANDRA-5633 exists for.
> As said above, 1) is already covered by collections, but 2) and 3) are not 
> (and
> I strongly believe collections are not the right fit, API wise, for those).
> Also, note that I don't want to underestimate the usefulness of 2). In most 
> cases, using a separate table for the blog posts and the comments is The 
> Right Solution, and trying to do 2) is premature optimisation. Yet, when used 
> properly, that kind of optimisation can make a difference, so I think having 
> a relatively native solution for it in CQL3 could make sense.
> Regarding 3), though CASSANDRA-5633 would provide one solution for it, I have 
> the feeling that static columns actually are a more natural approach (in term 
> of API). That's arguably more of a personal opinion/feeling though.
> So long story short, CQL3 lacks a way to mix both some "static" and "dynamic" 
> rows in the same partition of the same CQL3 table, and I think such a tool 
> could have it's use.
> The proposal is thus to allow "static" columns. Static columns would only 
> make sense in table with clustering columns (the "dynamic" ones). A static 
> column value would be static to the partition (all rows of the partition 
> would share the value for such column). The syntax would just be:
> {noformat}
> CREATE TABLE t (
>   k text,
>   s text static,
>   i int,
>   v text,
>   PRIMARY KEY (k, i)
> )
> {noformat}
> then you'd get:
> {noformat}
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm shared",   0, "foo");
> INSERT INTO t(k, s, i, v) VALUES ("k0", "I'm still shared", 1, "bar");
> SELECT * FROM t;
>  k |  s | i |v
> 
> k0 | "I'm still shared" | 0 | "bar"
> k0 | "I'm still shared" | 1 | "foo"
> {noformat}
> There would be a few semantic details to decide on regarding deletions, ttl, 
> etc. but let's see if we agree it's a good idea first before ironing those 
> out.
> One last point is the implementation. Though I do think this idea has merits, 
> it's definitively not useful enough to justify rewriting the storage engine 
> for it. But I think we can support this relatively easily (emphasis on 
> "relatively" :)), which is probably the main reason why I like the approach.
> Namely, internally, we can store static columns as cells whose clustering 
> column values are empty. So in terms of cells, the partition of my example 
> would look like:
> {noformat}
> "k0" : [
>   (:"s" -> "I'm still shared"), // the static column
>   (0:"" -> "")  // row marker
>   (0:"v" -> "bar")
>   (1:"" -> "")  

[jira] [Created] (CASSANDRA-6584) LOCAL_SERIAL doesn't work from Thrift

2014-01-14 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6584:
--

 Summary: LOCAL_SERIAL doesn't work from Thrift
 Key: CASSANDRA-6584
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6584
 Project: Cassandra
  Issue Type: Bug
  Components: API
Reporter: Nicolas Favre-Felix


Calling "cas" from Thrift with CL.LOCAL_SERIAL fails with an AssertionError 
since ThriftConversion.fromThrift has no "case" statement for LOCAL_SERIAL.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (CASSANDRA-6412) Custom creation and merge functions for user-defined column types

2013-12-04 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13839028#comment-13839028
 ] 

Nicolas Favre-Felix commented on CASSANDRA-6412:


Thanks for the feedback, [~slebresne].

I like your suggestion to use user-defined types, this is definitely better 
than the home-made candlestick structure.
I also like that having fixed types with custom resolver makes it easier to 
write type-safe code with minimal changes to the Cassandra code base.

As you point out, we can use the same technique as for counter deletion. I 
understand that counter deletes are somewhat broken, and that columns with a 
custom resolver would suffer from a similar defect (CASSANDRA-2774).

I don't think that there is an easy solution to this problem; only deleting al 
CL.ALL would prevent old values from being merged with newer ones.

> Custom creation and merge functions for user-defined column types
> -
>
> Key: CASSANDRA-6412
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6412
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Core
>Reporter: Nicolas Favre-Felix
>
> This is a proposal for a new feature, mapping custom types to Cassandra 
> columns.
> These types would provide a creation function and a merge function, to be 
> implemented in Java by the user.
> This feature relates to the concept of CRDTs; the proposal is to replicate 
> "operations" on these types during write, to apply these operations 
> internally during merge (Column.reconcile), and to also merge their values on 
> read.
> The following operations are made possible without reading back any data:
> * MIN or MAX(value) for a column
> * First value for a column
> * Count Distinct
> * HyperLogLog
> * Count-Min
> And any composition of these too, e.g. a Candlestick type includes first, 
> last, min, and max.
> The merge operations exposed by these types need to be commutative; this is 
> the case for many functions used in analytics.
> This feature is incomplete without some integration with CASSANDRA-4775 
> (Counters 2.0) which provides a Read-Modify-Write implementation for 
> distributed counters. Integrating custom creation and merge functions with 
> new counters would let users implement complex CRDTs in Cassandra, including:
> * Averages & related (sum of squares, standard deviation)
> * Graphs
> * Sets
> * Custom registers (even with vector clocks)
> I have a working prototype with implementations for min, max, and Candlestick 
> at https://github.com/acunu/cassandra/tree/crdts - I'd appreciate any 
> feedback on the design and interfaces.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6412) Custom creation and merge functions for user-defined column types

2013-11-27 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6412:
--

 Summary: Custom creation and merge functions for user-defined 
column types
 Key: CASSANDRA-6412
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6412
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Nicolas Favre-Felix


This is a proposal for a new feature, mapping custom types to Cassandra columns.
These types would provide a creation function and a merge function, to be 
implemented in Java by the user.
This feature relates to the concept of CRDTs; the proposal is to replicate 
"operations" on these types during write, to apply these operations internally 
during merge (Column.reconcile), and to also merge their values on read.

The following operations are made possible without reading back any data:
* MIN or MAX(value) for a column
* First value for a column
* Count Distinct
* HyperLogLog
* Count-Min

And any composition of these too, e.g. a Candlestick type includes first, last, 
min, and max.

The merge operations exposed by these types need to be commutative; this is the 
case for many functions used in analytics.

This feature is incomplete without some integration with CASSANDRA-4775 
(Counters 2.0) which provides a Read-Modify-Write implementation for 
distributed counters. Integrating custom creation and merge functions with new 
counters would let users implement complex CRDTs in Cassandra, including:

* Averages & related (sum of squares, standard deviation)
* Graphs
* Sets
* Custom registers (even with vector clocks)

I have a working prototype with implementations for min, max, and Candlestick 
at https://github.com/acunu/cassandra/tree/crdts - I'd appreciate any feedback 
on the design and interfaces.




--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6201) Expose compression metadata in cfstats

2013-10-15 Thread Nicolas Favre-Felix (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-6201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nicolas Favre-Felix updated CASSANDRA-6201:
---

Attachment: cassandra-trunk-6201.patch

> Expose compression metadata in cfstats
> --
>
> Key: CASSANDRA-6201
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6201
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Tools
>Reporter: Nicolas Favre-Felix
>Priority: Minor
>  Labels: jmx, nodetool
> Attachments: cassandra-trunk-6201.patch
>
>
> It would be useful to expose how much off-heap memory is allocated for 
> compression metadata.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6201) Expose compression metadata in cfstats

2013-10-15 Thread Nicolas Favre-Felix (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-6201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nicolas Favre-Felix updated CASSANDRA-6201:
---

Attachment: (was: cassandra-trunk-6201.patch)

> Expose compression metadata in cfstats
> --
>
> Key: CASSANDRA-6201
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6201
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Tools
>Reporter: Nicolas Favre-Felix
>Priority: Minor
>  Labels: jmx, nodetool
> Attachments: cassandra-trunk-6201.patch
>
>
> It would be useful to expose how much off-heap memory is allocated for 
> compression metadata.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6201) Expose compression metadata in cfstats

2013-10-15 Thread Nicolas Favre-Felix (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-6201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nicolas Favre-Felix updated CASSANDRA-6201:
---

Attachment: cassandra-trunk-6201.patch

Patch is for trunk.

I'm not sure about the safety of looping over the set of sstables - advice 
welcome.

Tested with compression enabled and disabled.

> Expose compression metadata in cfstats
> --
>
> Key: CASSANDRA-6201
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6201
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Tools
>Reporter: Nicolas Favre-Felix
>Priority: Minor
>  Labels: jmx, nodetool
> Attachments: cassandra-trunk-6201.patch
>
>
> It would be useful to expose how much off-heap memory is allocated for 
> compression metadata.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6201) Expose compression metadata in cfstats

2013-10-15 Thread Nicolas Favre-Felix (JIRA)
Nicolas Favre-Felix created CASSANDRA-6201:
--

 Summary: Expose compression metadata in cfstats
 Key: CASSANDRA-6201
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6201
 Project: Cassandra
  Issue Type: New Feature
  Components: Tools
Reporter: Nicolas Favre-Felix
Priority: Minor


It would be useful to expose how much off-heap memory is allocated for 
compression metadata.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-4775) Counters 2.0

2013-10-01 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13782998#comment-13782998
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4775:


I realize that there hasn't been much progress on this ticket during the 
summer. Following [~jbellis]'s call for contributors on cassandra-dev, we 
(Acunu) are willing to dedicate more time and resources to this particular 
ticket to see it implemented in the near future.

[~iamaleksey], thanks for your comments and critique; I would like to summarize 
the above and hopefully move this discussion towards a design that we can agree 
on.

As you pointed out, the set-based approach has some significant drawbacks:
* The design is complex and the distributed problem of merging deltas pushes 
some of this complexity to the client side.
* Reads are always more expensive as all replicas need to return the set of 
deltas they know about.
* The unpredictability in the distribution of read latencies is a serious issue.

The set-based design also has two advantages, speed (skipping a random read) 
and idempotence; I'd like to address these two below.

Although the read on the write path is an anti-pattern for Cassandra, it serves 
a useful purpose and ensures that the data sent from the coordinator to the 
replicas is always already summed up which means that reads stay predictably 
fast. The read during replicate_on_write is also more expensive than the one in 
a RMW design since we have to look up all the counter shards in (potentially) 
many sstables, and we have to do this every single time we increment a counter. 
RMW counters in Riak and HBase only need to read the latest value for the 
counter, and can immediately write it back to an in-memory data structure. I 
would also like to point out that reads will only become cheaper in the future 
as solid state drives become more commonplace, and not by a small amount. 
Designing a complex and unreliable solution to address a disappearing problem 
would be a mistake.

Idempotence is also a very useful property and the lack of safety for Cassandra 
counters is probably the first drawback that people mention when they describe 
Cassandra counters. Their reliability is questioned pretty often, and this 
criticism is not without merit. [~slebresne]'s suggestion of a retryable 
increment with a lock around the commit log entry is a great improvement over 
the current design, with the only limitation that the operation has to be 
commutative. I have written above that I had my doubts about the throughput of 
counters with such a lock, but I also recognize that there could be more 
optimisations as some suggested.

Internally, I would suggest the sharded counter design remain similar with one 
shard per replica for all commutative replicated data types; implementing 
counters as PN-counters is also a great way to introduce more general data 
types, something I believe would be much welcomed by the Cassandra community.

After this long discussion I believe the following design would be a viable 
alternative:
* Remove the replicate_on_write option and switch to a locked/retryable RMW.
* Provide a pluggable way to implement commutative types, with a PN-counter 
implementation.

This solution could also be migrated to from an existing deployment; the 
proposed set-based design would be too complex for that.

We are ready to work on the implementation as well in time for the target 
release of 2.1.

> Counters 2.0
> 
>
> Key: CASSANDRA-4775
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4775
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Core
>Reporter: Arya Goudarzi
>Assignee: Aleksey Yeschenko
>  Labels: counters
> Fix For: 2.1
>
>
> The existing partitioned counters remain a source of frustration for most 
> users almost two years after being introduced.  The remaining problems are 
> inherent in the design, not something that can be fixed given enough 
> time/eyeballs.
> Ideally a solution would give us
> - similar performance
> - less special cases in the code
> - potential for a retry mechanism



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-4775) Counters 2.0

2013-07-24 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13718481#comment-13718481
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4775:


[~jbellis] sorry about the late answer.

I am not convinced that locking before to the commit log entry is a great idea.
First, it does not *solve* the retry problem, even if it does mitigate it 
somewhat. It allows batches to be retried internally but doesn't give any 
guarantee to the client in the case of a timeout before the batch is added to 
the batchlog.
I implemented a read-modify-write (RMW) counter as a personal exercise last 
year and gave up on the idea because its performance was much lower than the 
current implementation. Cassandra currently allows concurrent updates to the 
same counter, with two clients applying deltas +x and +y, resulting in two 
replication reads that might both read (+x+y). This is not possible with a 
locked RMW and I remember observing many more timeouts on "hot" counters due to 
contention on this very coarse lock.
My toy implementation did not even lock around the commit log entry, which 
would be even slower.

It is true that the read in a RMW design is cheaper than the current read which 
might be touching several SSTables, but it's still very expensive and I'm 
worried that the internal retry safety wouldn't be enough to convince users 
that these slower counters are "better".

What do you think?

> Counters 2.0
> 
>
> Key: CASSANDRA-4775
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4775
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Core
>Reporter: Arya Goudarzi
>Assignee: Aleksey Yeschenko
>  Labels: counters
> Fix For: 2.1
>
>
> The existing partitioned counters remain a source of frustration for most 
> users almost two years after being introduced.  The remaining problems are 
> inherent in the design, not something that can be fixed given enough 
> time/eyeballs.
> Ideally a solution would give us
> - similar performance
> - less special cases in the code
> - potential for a retry mechanism

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4775) Counters 2.0

2013-07-05 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13700953#comment-13700953
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4775:


A few comments on the design posted above in a GitHub gist:

* The "time" part of the client-provided TimeUUID is now compared to the 
server's timestamp in the test "if(time(update-timeuuid) < now() - 
counter_write_window)". This is not ideal in my opinion, but I guess Cassandra 
is now using "real" timestamps a lot more than it used to. In any case, an 
"old" delta could also fall behind a "merge" cell and be ignored on read.
* Having "merge cells" means that we could support both TTLs and "put" 
operations on counters, as long as the semantics are well defined.
* Could counter merges happen in the background at ALL since most reads will 
receive responses from all replicas anyway, or would we always require QUORUM 
writes? This could be too restrictive in multi-DC deployments where most people 
probably prefer to read and write at LOCAL_QUORUM.


As described above, finding a "merge point" at which we could roll-up deltas 
involves either QUORUM reads + QUORUM writes or a read at ALL. This is 
necessary since we need a majority of replicas to persist the merge cell. We 
could consider this "set of deltas" that make up a counter to be merged at 
different levels, though. When this set is common to all replicas (as is 
proposed above), we can only merge in QUORUM reads if we can _guarantee_ quorum 
writes or must merge in reads at ALL otherwise.

If, instead, we shard this "set of deltas" among replicas with a distribution 
scheme resembling the existing implementation, each replica becomes the 
"coordinator" for a fraction of deltas and can (on its own) merge the 
increments for which it is responsible and issue "merge cells" to replace them. 
It becomes possible to read and write at LOCAL_QUORUM using this scheme. As any 
particular replica is the only source of truth for the subset of deltas that it 
was assigned, it does _by definition_ read ALL of its deltas and can sum them 
up with no risk of inconsistency. When these cells are node-local with a single 
source of truth, they can be merged by their owner and a merge cell replicated 
easily.
The main issue with this implementation is the choice of the coordinator node 
for an increment operation: if we assign a replica at random, retrying would 
lead to duplicates; if we assign a replica deterministically (based on the 
operation's UUID for example) we risk not being able to write to the counter if 
that particular replica goes down.

I'd like to propose a solution that lies between merging counters across the 
whole cluster and merging counters in each individual replica:
We can shard counters based on the datacenter, and roll-up these UUIDs per DC. 
In that case, the scope of the set of replicas involved in merging deltas 
together is therfore limited to the replicas of the local DC, which (once 
again) can merge deltas by either getting involved at W.QUORUM+R.QUORUM or 
W.anything+R.ALL.
A configuration flag per counter CF would configure whether we require 
W.QUORUM+R.QUORUM (default) or let clients write with any CL with the downside 
that deltas can only be merged at CL.ALL.
The same issue with retries applies here, albeit at a different level: a 
particular operation can only be retried safely if it sent to the same 
datacenter, which seems reasonable.

I believe that the space and time overheads are about the same as in Aleksey's 
design.

Suggestions and ideas much welcome.


> Counters 2.0
> 
>
> Key: CASSANDRA-4775
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4775
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Core
>Reporter: Arya Goudarzi
>Assignee: Aleksey Yeschenko
>  Labels: counters
> Fix For: 2.1
>
>
> The existing partitioned counters remain a source of frustration for most 
> users almost two years after being introduced.  The remaining problems are 
> inherent in the design, not something that can be fixed given enough 
> time/eyeballs.
> Ideally a solution would give us
> - similar performance
> - less special cases in the code
> - potential for a retry mechanism

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4775) Counters 2.0

2013-04-05 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13623757#comment-13623757
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4775:


I would like to describe a design that was discussed at Acunu last year, aiming 
to resolve the problems pointed out by Sylvain as well as remove the read 
operation needed by replicate_on_write.

Our solution added a unique identifier per counter update operation, used to 
identify duplicate commands and avoid overcounts on retry. The main problem in 
storing (UUID, delta) pairs per counter is the O(N) read complexity; this is 
how people implemented counters before 0.8 and it is a pretty inefficient way 
of counting things.

Our idea was to merge those update pairs in the back-end, trying to always keep 
a small number of deltas instead of all of them. Merging those updates requires 
some level of synchronisation between the replicas, but that's not something 
that Cassandra is completely adverse to as active anty-entropy also requires 
all replicas to be available.
This design considered using a tree per counter, with time-based buckets 
containing all increments to the counter for a given time period - say, 5 
seconds by default. Once this time has passed, the bucket for the past 5 
seconds is queued for synchronization amongst all replicas and eventually 
replaced with an equivalent bucket containing a single increment with a UUID 
built from all the updates that it replaces (using XOR would work). If the 
replicas disagree on what needs to be in the bucket, they send each other 
missed updates in the same way that data is exchanged during repair. If a node 
is down, we keep accumulating 5-second buckets that will need to be merged 
later.
The 5-second buckets are eventually merged into a minute bucket, then an hour 
bucket, etc.

As an added bonus, the reduce function can be set to MIN, MAX, SUM_SQ, etc. 
instead of just SUM.

Here are the main drawbacks I see for this approach:

* The implementation becomes a fair bit more complicated.
* Counters take more space that they used to.
* The replicas need to all be up for the "collapsing" operation to run. It 
might just be that counters start to get slower if some of your nodes are down 
for a long time. You can't merge updates with a replica down or you might lose 
increments.
* We introduce an actual timestamp instead of the current binary blob.
* The implementation is not compatible with the current one.
* The performance characteristics of these counters are unknown.
* No code exists.


> Counters 2.0
> 
>
> Key: CASSANDRA-4775
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4775
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Core
>Reporter: Arya Goudarzi
>Assignee: Aleksey Yeschenko
>  Labels: counters
> Fix For: 2.0
>
>
> The existing partitioned counters remain a source of frustration for most 
> users almost two years after being introduced.  The remaining problems are 
> inherent in the design, not something that can be fixed given enough 
> time/eyeballs.
> Ideally a solution would give us
> - similar performance
> - less special cases in the code
> - potential for a retry mechanism

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4482) In-memory merkle trees for repair

2012-08-15 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434880#comment-13434880
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4482:


bq. What is a tombstone-only repair?

This is a separate feature that we provide, that runs the AES on tombstones 
exclusively. We provide it for users who repair in order to avoid reappearing 
tombstones rather that to reduce entropy. The main point is that repair and 
tombstone repair are both guaranteed to result in a synchronized cluster, which 
is not the case for incremental repair.

Thanks for the review offer, I will extract and clean up the patch and submit 
it to this ticket.

> In-memory merkle trees for repair
> -
>
> Key: CASSANDRA-4482
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4482
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>
> this sounds cool, we should reimplement it in the open source cassandra;
> http://www.acunu.com/2/post/2012/07/incremental-repair.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4482) In-memory merkle trees for repair

2012-08-14 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434267#comment-13434267
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4482:


A node wouldn't have missed A because of a compaction but because A was not 
inserted there, since S is not built from the existing data on disk but 
incrementally with each change: S really represents the _combined history of 
all the changes_ performed on the token range since the last repair session.
So nodes don't have to scan their data to build S, they simply start with S=0 
when incremental repair is first enabled regardless of their initial 
differences and start again with S=0 after each incremental repair session.

But it is indeed possible for two replicas to have the same data but differing 
values for S, for instance if a replica gets A and A' whereas another misses A 
but gets A': this would lead to some unnecessary streaming even though they 
both have the latest value A'. This could be avoided by removing A from S as 
you suggest, but the cost of doing random I/O after each write is too 
expensive, as you pointed out earlier.

We are open to suggestions on how to improve this process and get this feature 
upstreamed with these issues addressed or understood as inherent limitations.

> In-memory merkle trees for repair
> -
>
> Key: CASSANDRA-4482
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4482
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>
> this sounds cool, we should reimplement it in the open source cassandra;
> http://www.acunu.com/2/post/2012/07/incremental-repair.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4482) In-memory merkle trees for repair

2012-08-14 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434171#comment-13434171
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4482:


Indeed, I've used "range" as Range, the range of tokens owned by a node; 
I should have made this clearer.
We are not using the MerkleTree class or its TreeRange objects, but updating a 
single ByteBuffer directly instead of creating the whole tree with its hundreds 
of internal objects. This is equivalent to updating the leaves alone, without 
propagating the hash upwards in the tree. Yes, that means comparing two trees 
is O(leaf count).

bq. I xor all these together to get my initial state, S. To update row A to row 
A', I need to take S xor hash(A) xor hash(A').

If you've lready xor'd all these together, S does include the hash of your 
existing row A. Updating A to A' hashes A' and returns S' = S xor hash(A'), 
which is hash(A') xor hash(A).

In practice, this is how it works step by step:

# Load existing buffers when the ColumnFamilyStore is created: per 
Range, load an existing buffer or create a new one initialized with 
zeros.
# ColumnFamilyStore.apply() is called with columns X and Y in row A. For 
instance, row A could have token 0x10, falling in the range (0x00, 0x20]. The 
incremental repair ByteBuffer for this range is 1 KB in size.
# Create a new digest and run Column.updateDigest() on X and Y sucessively. We 
end up with H = hash(X) xor hash(Y); H is 16 bytes long.
# Calculate O, the offset in the ByteBuffer that corresponds to H: in this 
case, it's around 512 since 0x10 is close to the middle of the range (0x00, 
0x20].
# For each byte i of H, we set buffer[O+i] = buffer[O+i] xor H[i].

During the repair session, the replicas send out their existing ByteBuffers for 
the range being repaired and replace them with empty ones that will receive 
subsequent inserts.

bq. And sync the BB saving with CF flushes so CL replay matches up, I imagine.

Yes. If you terminate Cassandra at this stage, the ByteBuffer is written to 
disk and will contains [0,0 a few bytes of hash(X) xor hash(Y) around the 
middle ... 0,0,0,0].


> In-memory merkle trees for repair
> -
>
> Key: CASSANDRA-4482
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4482
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>
> this sounds cool, we should reimplement it in the open source cassandra;
> http://www.acunu.com/2/post/2012/07/incremental-repair.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4482) In-memory merkle trees for repair

2012-08-14 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434105#comment-13434105
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4482:


bq. Meaning, you give each CF less than 64k ranges * 16 bytes / range?

Right, that would be too much. At the moment, we give each CF 256 KB to be 
split into all of its ranges. For num_tokens=256, that's 1 KB per range on 
average - we do not yet scale this number according to the range size.

A node with num_tokens = 1 owning a single range would allocate 256 KB in a 
single direct ByteBuffer. Moving to num_tokens = 256 gives the 
ColumnFamilyStore 256 ranges, and allocates a 1 KB ByteBuffer per range. In 
both cases the keys in any given range are covered by as many "leaf bytes" on 
average, regardless of the number of ranges.

bq. Is there a startup cost associated with the approach? i.e. How to you know 
the initial hash?

We do have to reload $num_tokens ByteBuffers when creating the 
ColumnFamilyStore, for a total of 256KB per CF with our current defaults. This 
is not something we've measured but I suspect that the cost is fairly small, as 
it is now for the cache snapshots: it is O(number of CFs), not O(N) like the 
old cache preloads.

> In-memory merkle trees for repair
> -
>
> Key: CASSANDRA-4482
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4482
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>
> this sounds cool, we should reimplement it in the open source cassandra;
> http://www.acunu.com/2/post/2012/07/incremental-repair.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4482) In-memory merkle trees for repair

2012-08-14 Thread Nicolas Favre-Felix (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434048#comment-13434048
 ] 

Nicolas Favre-Felix commented on CASSANDRA-4482:


Jonathan,

I wrote the blog post linked in this ticket; the incremental repair process 
we've implemented is not doing any random I/O on insert as you suggest.

Instead, we maintain a Merkle Tree (MT) in memory and update it with every 
single column insert in ColumnFamilyStore.apply(). We use 
column.updateDigest(digest) on all the changes in order to create a hash per 
column update and then XOR this hash with the existing one in the Merkle Tree 
bucket for the corresponding row.
This Merkle Tree is created with the column family (one per range), initialized 
with zeros, and persisted to disk with regular snapshots.
The commutative properties of XOR make it possible to update the MT 
incrementally without having to read on write.

When an incremental repair session starts, the CFS swap out their existing MTs 
for new empty ones that will receive subsequent updates.

There are a few downsides to this approach:
* It is possible for the incremental MTs to miss a few inserts that happen when 
the replicas involved swap out their MTs for new ones. An insert will be in the 
previous MT for node "A" but in the fresh one for node "B", for instance. This 
leads to either a very small amount of extra streaming or some unrepaired 
changes. For this reason, we still recommend that users run either a full 
repair or a "tombstone-only repair" at least once every GCGraceSeconds.
* There is some overhead to keeping these MTs in memory. We actually maintain 
only the leaves as a single ByteBuffer instead of creating all the intermediate 
nodes like the MerkleTree class does. To avoid using too much RAM, we allocate 
a fixed amount of memory per CF and divide it into a number of smaller buffers 
(one per range) in order to give the same guarantees regardless of the number 
of ranges per CF.
* There is a small cost in insert, about half of which is due to the hash 
function (MD5).

We are looking into making our patch available to the community and would 
welcome suggestions to solve or improve on these limitations.

> In-memory merkle trees for repair
> -
>
> Key: CASSANDRA-4482
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4482
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>
> this sounds cool, we should reimplement it in the open source cassandra;
> http://www.acunu.com/2/post/2012/07/incremental-repair.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira