[jira] [Created] (CASSANDRA-6129) get java.util.ConcurrentModificationException while bulkloading from sstable for widerow table

2013-10-02 Thread koray sariteke (JIRA)
koray sariteke created CASSANDRA-6129:
-

 Summary: get java.util.ConcurrentModificationException while 
bulkloading from sstable for widerow table
 Key: CASSANDRA-6129
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6129
 Project: Cassandra
  Issue Type: Bug
  Components: Core, Tools
 Environment: three cassandra 2.0.1 node
jdk 7
linux - ubuntu
Reporter: koray sariteke


I haven't faced that problem with cassandra 1.2.6

I have created widerow sstables with SSTableSimpleUnsortedWriter. When i tried 
to load sstables by sstableloader, I got 
java.util.ConcurrentModificationException after a while (not at the beggining 
of the streaming).

Exception is :
progress: [/192.168.103.5 0/39 (0%)] [/192.168.103.3 0/39 (0%)] [/192.168.103.1 
0/39 (0%)] [total: 0% - 15MB/s (avg: 0MB/s)] INFO 00:45:23,542 [Stream 
#c0f53e00-2ae2-11e3-ab6b-99a3e9e32246] Session with /192.168.103.3 is complete
progress: [/192.168.103.5 0/39 (0%)] [/192.168.103.3 0/39 (0%)] [/192.168.103.1 
0/39 (0%)] [total: 0% - 3MB/s (avg: 1MB/s)]Exception in thread 
STREAM-OUT-/192.168.103.3 java.util.ConcurrentModificationException
at java.util.HashMap$HashIterator.nextEntry(HashMap.java:894)
at java.util.HashMap$EntryIterator.next(HashMap.java:934)
at java.util.HashMap$EntryIterator.next(HashMap.java:932)
at 
org.apache.cassandra.tools.BulkLoader$ProgressIndicator.handleStreamEvent(BulkLoader.java:129)
at 
org.apache.cassandra.streaming.StreamResultFuture.fireStreamEvent(StreamResultFuture.java:198)
at 
org.apache.cassandra.streaming.StreamResultFuture.handleProgress(StreamResultFuture.java:191)
at 
org.apache.cassandra.streaming.StreamSession.progress(StreamSession.java:474)
at 
org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:105)
at 
org.apache.cassandra.streaming.messages.FileMessage$1.serialize(FileMessage.java:73)
at 
org.apache.cassandra.streaming.messages.FileMessage$1.serialize(FileMessage.java:45)
at 
org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44)
at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:384)
at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:357)
at java.lang.Thread.run(Thread.java:781)
progress: [/192.168.103.5 0/39 (3%)] [/192.168.103.3 0/39 (0%)] [/192.168.103.1 
0/39 (2%)] [total: 1% - 2147483647MB/s (avg: 12MB/s)]Exception in thread 
STREAM-OUT-/192.168.103.1 java.util.ConcurrentModificationException
at java.util.HashMap$HashIterator.nextEntry(HashMap.java:894)
at java.util.HashMap$KeyIterator.next(HashMap.java:928)
progress: [/192.168.103.5 0/39 (3%)] [/192.168.103.3 0/39 (0%)] [/192.168.103.1 
0/39 (2%)] [total: 1% - 2147483647MB/s (avg: 12MB/s)]
at 
org.apache.cassandra.streaming.StreamResultFuture.fireStreamEvent(StreamResultFuture.java:198)
at 
org.apache.cassandra.streaming.StreamResultFuture.handleProgress(StreamResultFuture.java:191)
at 
org.apache.cassandra.streaming.StreamSession.progress(StreamSession.java:474)
at 
org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:105)
at 
org.apache.cassandra.streaming.messages.FileMessage$1.serialize(FileMessage.java:73)
at 
org.apache.cassandra.streaming.messages.FileMessage$1.serialize(FileMessage.java:45)
at 
org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44)
at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:384)
at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:357)
at java.lang.Thread.run(Thread.java:781)




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


[jira] [Created] (CASSANDRA-6130) Secondary Index does not seems work

2013-10-02 Thread koray sariteke (JIRA)
koray sariteke created CASSANDRA-6130:
-

 Summary: Secondary Index does not seems work
 Key: CASSANDRA-6130
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6130
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: cassandra 2.0.1
jdk 7
Reporter: koray sariteke


When secondary index is created, not able to query by created index. We 
searched logs and not noticed any info about index. 



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


[jira] [Updated] (CASSANDRA-6130) Secondary Index does not work properly

2013-10-02 Thread koray sariteke (JIRA)

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

koray sariteke updated CASSANDRA-6130:
--

Summary: Secondary Index does not work properly  (was: Secondary Index does 
not seems work)

 Secondary Index does not work properly
 --

 Key: CASSANDRA-6130
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6130
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: cassandra 2.0.1
 jdk 7
Reporter: koray sariteke

 When secondary index is created, not able to query by created index. We 
 searched logs and not noticed any info about index. 



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


[jira] [Commented] (CASSANDRA-6130) Secondary Index does not work properly

2013-10-02 Thread koray sariteke (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783695#comment-13783695
 ] 

koray sariteke commented on CASSANDRA-6130:
---

can compression and compaction parameter play a role in that problem.

 Secondary Index does not work properly
 --

 Key: CASSANDRA-6130
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6130
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: cassandra 2.0.1
 jdk 7
Reporter: koray sariteke

 When secondary index is created, not able to query by created index. We 
 searched logs and not noticed any info about index. 



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


[jira] [Updated] (CASSANDRA-5981) Netty frame length exception when storing data to Cassandra using binary protocol

2013-10-02 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne updated CASSANDRA-5981:


Attachment: 5981-v2.txt

Alright, attaching a v2 (that includes making the max frame length 
configurable patch) that rewrite the Frame decoder to handle the frame 
slightly more manually to allow us to do what we want. This mostly mimick the 
code of Netty LengthFieldBasedFrameDecoder, though a bit simplified since 
adapted to just what we need. I'll note that this patch is against the 2.0 
branch: I've been able to run the java driver tests with that patch so we 
should be good but this still is not entirely trivial a change so I'm starting 
to wonder if it's worth pushing it in 1.2, especially given that the current 
behavior (having the error logged server side) is not really a big deal.

 Netty frame length exception when storing data to Cassandra using binary 
 protocol
 -

 Key: CASSANDRA-5981
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5981
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Linux, Java 7
Reporter: Justin Sweeney
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.2.11

 Attachments: 0001-Correctly-catch-frame-too-long-exceptions.txt, 
 0002-Allow-to-configure-the-max-frame-length.txt, 5981-v2.txt


 Using Cassandra 1.2.8, I am running into an issue where when I send a large 
 amount of data using the binary protocol, I get the following netty exception 
 in the Cassandra log file:
 {quote}
 ERROR 09:08:35,845 Unexpected exception during request
 org.jboss.netty.handler.codec.frame.TooLongFrameException: Adjusted frame 
 length exceeds 268435456: 292413714 - discarded
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.fail(LengthFieldBasedFrameDecoder.java:441)
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.failIfNecessary(LengthFieldBasedFrameDecoder.java:412)
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.decode(LengthFieldBasedFrameDecoder.java:372)
 at org.apache.cassandra.transport.Frame$Decoder.decode(Frame.java:181)
 at 
 org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:422)
 at 
 org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303)
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
 at 
 org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:84)
 at 
 org.jboss.netty.channel.socket.nio.AbstractNioWorker.processSelectedKeys(AbstractNioWorker.java:472)
 at 
 org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:333)
 at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:35)
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
 at java.lang.Thread.run(Thread.java:722)
 {quote}
 I am using the Datastax driver and using CQL to execute insert queries. The 
 query that is failing is using atomic batching executing a large number of 
 statements (~55).
 Looking into the code a bit, I saw that in the 
 org.apache.cassandra.transport.Frame$Decoder class, the MAX_FRAME_LENGTH is 
 hard coded to 256 mb.
 Is this something that should be configurable or is this a hard limit that 
 will prevent batch statements of this size from executing for some reason?



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


[jira] [Commented] (CASSANDRA-6119) IndexedSliceReader can skip columns when fetching multiple contiguous slices

2013-10-02 Thread Fabien Rousseau (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783806#comment-13783806
 ] 

Fabien Rousseau commented on CASSANDRA-6119:


After having reviewed your patch, it has definitely a better approach 
minimising code change and better readability

So I'm +1 for your patch

 IndexedSliceReader can skip columns when fetching multiple contiguous slices
 

 Key: CASSANDRA-6119
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6119
 Project: Cassandra
  Issue Type: Bug
Reporter: Fabien Rousseau
Assignee: Fabien Rousseau
 Attachments: 6119.patch, 6119-v2.txt


 This was observed using SliceQueryFilter with multiple slices.
 Let's have a row a having the following column list : colA, colB, 
 colC, colD
 Then select 2 ranges : [colA, colB], [colC, colD]
 Expected result is the four columns
 But only 3 are returned (colA, colB, colD)
 To reproduce the above scenario in the unit tests, you can modify the test 
 ColumnFamilyStoreTest.testMultiRangeIndexed by replacing the original line :
 String[] letters = new String[] { a, b, c, d, e, f, g, 
 h, i };
 by this one (f letter has been removed) :
 String[] letters = new String[] { a, b, c, d, e, g, h, 
 i };
 Anyway, a patch is attached which adds more unit tests, and modifies 
 IndexedSliceReader.IndexedBlockFetcher  
 IndexedSliceReader.SimpleBlockFetcher 



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


[jira] [Updated] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-10-02 Thread Benedict (JIRA)

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

Benedict updated CASSANDRA-4718:


Attachment: op costs of various queues.ods

 More-efficient ExecutorService for improved throughput
 --

 Key: CASSANDRA-4718
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Priority: Minor
  Labels: performance
 Attachments: baq vs trunk.png, op costs of various queues.ods, 
 PerThreadQueue.java


 Currently all our execution stages dequeue tasks one at a time.  This can 
 result in contention between producers and consumers (although we do our best 
 to minimize this by using LinkedBlockingQueue).
 One approach to mitigating this would be to make consumer threads do more 
 work in bulk instead of just one task per dequeue.  (Producer threads tend 
 to be single-task oriented by nature, so I don't see an equivalent 
 opportunity there.)
 BlockingQueue has a drainTo(collection, int) method that would be perfect for 
 this.  However, no ExecutorService in the jdk supports using drainTo, nor 
 could I google one.
 What I would like to do here is create just such a beast and wire it into (at 
 least) the write and read stages.  (Other possible candidates for such an 
 optimization, such as the CommitLog and OutboundTCPConnection, are not 
 ExecutorService-based and will need to be one-offs.)
 AbstractExecutorService may be useful.  The implementations of 
 ICommitLogExecutorService may also be useful. (Despite the name these are not 
 actual ExecutorServices, although they share the most important properties of 
 one.)



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


[jira] [Comment Edited] (CASSANDRA-6130) Secondary Index does not work properly

2013-10-02 Thread koray sariteke (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783695#comment-13783695
 ] 

koray sariteke edited comment on CASSANDRA-6130 at 10/2/13 10:45 AM:
-

can compression and compaction parameter play a role in that problem?


was (Author: ksaritek):
can compression and compaction parameter play a role in that problem.

 Secondary Index does not work properly
 --

 Key: CASSANDRA-6130
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6130
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: cassandra 2.0.1
 jdk 7
Reporter: koray sariteke

 When secondary index is created, not able to query by created index. We 
 searched logs and not noticed any info about index. 



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


[jira] [Commented] (CASSANDRA-5658) TracingStage frequently times out

2013-10-02 Thread Jeremy Hanna (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783846#comment-13783846
 ] 

Jeremy Hanna commented on CASSANDRA-5658:
-

A user that I've been working with has seen this as well - the exact exception 
in the description.  Just as another data point.

 TracingStage frequently times out
 -

 Key: CASSANDRA-5658
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5658
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.2.4, 1.2.6, 2.0 beta 1
Reporter: Ryan McGuire
 Attachments: 5658-logs.tar.gz, trace_bug.cql, trace_bug_cqlsh.py, 
 trace_bug.py


 I am seeing frequent timeout errors when doing programmatic traces via 
 trace_next_query()
 {code}
 ERROR [TracingStage:1] 2013-06-18 19:10:20,669 CassandraDaemon.java (line 
 196) Exception in thread Thread[TracingStage:1,5,main]
 java.lang.RuntimeException: 
 org.apache.cassandra.exceptions.WriteTimeoutException: Operation timed out - 
 received only 0 responses.
 at com.google.common.base.Throwables.propagate(Throwables.java:160)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
 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:722)
 Caused by: org.apache.cassandra.exceptions.WriteTimeoutException: Operation 
 timed out - received only 0 responses.
 at 
 org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:81)
 at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:454)
 at 
 org.apache.cassandra.tracing.TraceState$1.runMayThrow(TraceState.java:100)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 ... 3 more
 {code}
 Attached is the sample code which produced this error and the logs. The error 
 occurs directly after the INSERT statement.



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


[jira] [Updated] (CASSANDRA-5981) Netty frame length exception when storing data to Cassandra using binary protocol

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-5981:
--

Fix Version/s: (was: 1.2.11)
   2.0.2

Agreed on the 2.0 call.

 Netty frame length exception when storing data to Cassandra using binary 
 protocol
 -

 Key: CASSANDRA-5981
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5981
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Linux, Java 7
Reporter: Justin Sweeney
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 2.0.2

 Attachments: 0001-Correctly-catch-frame-too-long-exceptions.txt, 
 0002-Allow-to-configure-the-max-frame-length.txt, 5981-v2.txt


 Using Cassandra 1.2.8, I am running into an issue where when I send a large 
 amount of data using the binary protocol, I get the following netty exception 
 in the Cassandra log file:
 {quote}
 ERROR 09:08:35,845 Unexpected exception during request
 org.jboss.netty.handler.codec.frame.TooLongFrameException: Adjusted frame 
 length exceeds 268435456: 292413714 - discarded
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.fail(LengthFieldBasedFrameDecoder.java:441)
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.failIfNecessary(LengthFieldBasedFrameDecoder.java:412)
 at 
 org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder.decode(LengthFieldBasedFrameDecoder.java:372)
 at org.apache.cassandra.transport.Frame$Decoder.decode(Frame.java:181)
 at 
 org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:422)
 at 
 org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303)
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
 at 
 org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:84)
 at 
 org.jboss.netty.channel.socket.nio.AbstractNioWorker.processSelectedKeys(AbstractNioWorker.java:472)
 at 
 org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:333)
 at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:35)
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
 at java.lang.Thread.run(Thread.java:722)
 {quote}
 I am using the Datastax driver and using CQL to execute insert queries. The 
 query that is failing is using atomic batching executing a large number of 
 statements (~55).
 Looking into the code a bit, I saw that in the 
 org.apache.cassandra.transport.Frame$Decoder class, the MAX_FRAME_LENGTH is 
 hard coded to 256 mb.
 Is this something that should be configurable or is this a hard limit that 
 will prevent batch statements of this size from executing for some reason?



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


[jira] [Commented] (CASSANDRA-4338) Experiment with direct buffer in SequentialWriter

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4338?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783895#comment-13783895
 ] 

Jonathan Ellis commented on CASSANDRA-4338:
---

Promising!

 Experiment with direct buffer in SequentialWriter
 -

 Key: CASSANDRA-4338
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4338
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Marcus Eriksson
Priority: Minor
  Labels: performance
 Fix For: 2.1

 Attachments: 4338-gc.tar.gz, gc-4338-patched.png, gc-trunk-me.png, 
 gc-trunk.png, gc-with-patch-me.png


 Using a direct buffer instead of a heap-based byte[] should let us avoid a 
 copy into native memory when we flush the buffer.



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


[jira] [Commented] (CASSANDRA-4338) Experiment with direct buffer in SequentialWriter

2013-10-02 Thread Marcus Eriksson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4338?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783883#comment-13783883
 ] 

Marcus Eriksson commented on CASSANDRA-4338:


So, got the CompressedSequentialWriter working, code pushed to github: 
https://github.com/krummas/cassandra/commits/marcuse/4338. It uses snappys 
direct bytebuffer support, and a custom Adler32 made by me that can checksum 
direct byte buffers (code here: https://github.com/krummas/adler32 (probably 
only builds on linux, did not spend much time on it).

Micro benchmarks look great, almost no GC at all with the patched version (the 
benchmark is left in main(...) in CompressedSequentialWriter.java):
.h2 Trunk
!gc-trunk-me.png!

.h2 Patched
!gc-with-patch-me.png!

Proper single-node stress benchmarks look good as well:
h2. Trunk
{noformat}
total,interval_op_rate,interval_key_rate,latency,95th,99.9th,elapsed_time
394141,39414,39414,0.0,0.0,0.0,10
1078321,68418,68418,0.0,0.0,0.0,20
1726219,64789,64789,0.0,0.0,0.0,30
2327295,60107,60107,0.0,0.0,0.0,40
2928533,60123,60123,0.0,0.0,0.0,50
3533878,60534,60534,0.0,0.0,0.0,60
3602168,6829,6829,0.0,0.0,0.0,70
3967820,36565,36565,0.0,0.0,0.0,80
4647217,67939,67939,0.0,0.0,0.0,91
5248142,60092,60092,0.0,0.0,0.0,101
5930662,68252,68252,0.0,0.0,0.0,111
6417903,48724,48724,0.0,0.0,0.0,121
6952933,53503,53503,0.0,0.0,0.0,131
7221662,26872,26872,0.0,0.0,0.0,141
7221662,0,0,0.0,0.0,0.0,151
7221662,0,0,0.0,0.0,0.0,161
7221662,0,0,0.0,0.0,0.0,172
7221662,0,0,0.0,0.0,0.0,182
7221662,0,0,0.0,0.0,0.0,192
7509240,28757,28757,0.0,0.0,0.0,202
7780984,27174,27174,0.0,0.0,0.0,212
7780984,0,0,0.0,0.0,0.0,222
7780984,0,0,0.0,0.0,0.0,232
7780984,0,0,0.0,0.0,0.0,242
8414140,63315,63315,0.0,0.0,0.0,252
8968246,55410,55410,0.0,0.0,0.0,263
9669857,70161,70161,0.0,0.0,0.0,273
10236467,56661,56661,0.0,0.0,0.0,283
10774593,53812,53812,0.0,0.0,0.0,293
10824657,5006,5006,0.0,0.0,0.0,303
11165174,34051,34051,0.0,0.0,0.0,313
11165174,0,0,0.0,0.0,0.0,323
11165174,0,0,0.0,0.0,0.0,333
11165174,0,0,0.0,0.0,0.0,343
11304248,13907,13907,0.0,0.0,0.0,354
11927380,62313,62313,0.0,0.0,0.0,364
12526960,59958,59958,0.0,0.0,0.0,374
13234647,70768,70768,0.0,0.0,0.0,384
13792652,55800,55800,0.0,0.0,0.0,394
14329718,53706,53706,0.0,0.0,0.0,404
14512350,18263,18263,0.0,0.0,0.0,414
14512929,57,57,0.0,0.0,0.0,424
14710476,19754,19754,0.0,0.0,0.0,434
14710476,0,0,0.0,0.0,0.0,445
14710476,0,0,0.0,0.0,0.0,455
15061043,35056,35056,0.0,0.0,0.0,465
15760509,69946,69946,0.0,0.0,0.0,475
16461318,70080,70080,0.0,0.0,0.0,485
17126749,66543,66543,0.0,0.0,0.0,495
17708154,58140,58140,0.0,0.0,0.0,505
18226801,51864,51864,0.0,0.0,0.0,515
18226801,0,0,0.0,0.0,0.0,526
18227225,42,42,0.0,0.0,0.0,536
18858228,63100,63100,0.0,0.0,0.0,546
19459047,60081,60081,0.0,0.0,0.0,556
19988583,52953,52953,0.0,0.0,0.0,566
2000,1141,1141,0.0,0.0,0.0,567


Averages from the middle 80% of values:
interval_op_rate  : 34003
interval_key_rate : 34003
latency median: 0.0
latency 95th percentile   : 0.0
latency 99.9th percentile : 0.0
Total operation time  : 00:09:27
END
{noformat}
h2. Patched version
{noformat}
total,interval_op_rate,interval_key_rate,latency,95th,99.9th,elapsed_time
398380,39838,39838,0.0,0.0,0.0,10
1090332,69195,69195,0.0,0.0,0.0,20
1756859,66652,66652,0.0,0.0,0.0,30
2408330,65147,65147,0.0,0.0,0.0,40
3021314,61298,61298,0.0,0.0,0.0,50
3602221,58090,58090,0.0,0.0,0.0,60
3602221,0,0,0.0,0.0,0.0,70
4086404,48418,48418,0.0,0.0,0.0,80
4670997,58459,58459,0.0,0.0,0.0,91
5328657,65766,65766,0.0,0.0,0.0,101
5950535,62187,62187,0.0,0.0,0.0,111
6544475,59394,59394,0.0,0.0,0.0,121
7163644,61916,61916,0.0,0.0,0.0,131
7307634,14399,14399,0.0,0.0,0.0,141
7331684,2405,2405,0.0,0.0,0.0,151
7989707,65802,65802,0.0,0.0,0.0,161
8653302,66359,66359,0.0,0.0,0.0,172
9273188,61988,61988,0.0,0.0,0.0,182
9935986,66279,66279,0.0,0.0,0.0,192
10489010,55302,55302,0.0,0.0,0.0,202
10909996,42098,42098,0.0,0.0,0.0,212
10962871,5287,5287,0.0,0.0,0.0,222
11274293,31142,31142,0.0,0.0,0.0,232
11274293,0,0,0.0,0.0,0.0,242
11274293,0,0,0.0,0.0,0.0,252
11297105,2281,2281,0.0,0.0,0.0,263
11946842,64973,64973,0.0,0.0,0.0,273
12509283,56244,56244,0.0,0.0,0.0,283
13205933,69665,69665,0.0,0.0,0.0,293
13809534,60360,60360,0.0,0.0,0.0,303
14334735,52520,52520,0.0,0.0,0.0,313
14615255,28052,28052,0.0,0.0,0.0,323
14615958,70,70,0.0,0.0,0.0,333
14841997,22603,22603,0.0,0.0,0.0,343
14841997,0,0,0.0,0.0,0.0,354
14841997,0,0,0.0,0.0,0.0,364
15262968,42097,42097,0.0,0.0,0.0,374
15943731,68076,68076,0.0,0.0,0.0,384
16619205,67547,67547,0.0,0.0,0.0,394
17197417,57821,57821,0.0,0.0,0.0,404
17776353,57893,57893,0.0,0.0,0.0,414
18235461,45910,45910,0.0,0.0,0.0,424
18267460,3199,3199,0.0,0.0,0.0,434
18592152,32469,32469,0.0,0.0,0.0,445
18732480,14032,14032,0.0,0.0,0.0,455
19328150,59567,59567,0.0,0.0,0.0,465
19930114,60196,60196,0.0,0.0,0.0,475
2000,6988,6988,0.0,0.0,0.0,479


Averages 

[jira] [Comment Edited] (CASSANDRA-4338) Experiment with direct buffer in SequentialWriter

2013-10-02 Thread Marcus Eriksson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4338?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13783883#comment-13783883
 ] 

Marcus Eriksson edited comment on CASSANDRA-4338 at 10/2/13 12:58 PM:
--

So, got the CompressedSequentialWriter working, code pushed to github: 
https://github.com/krummas/cassandra/commits/marcuse/4338. It uses snappys 
direct bytebuffer support, and a custom Adler32 made by me that can checksum 
direct byte buffers (code here: https://github.com/krummas/adler32 (probably 
only builds on linux, did not spend much time on it).

Micro benchmarks look great, almost no GC at all with the patched version (the 
benchmark is left in main(...) in CompressedSequentialWriter.java):

.h2 Trunk
!gc-trunk-me.png!

.h2 Patched
!gc-with-patch-me.png!

Proper single-node stress benchmarks look good as well:
h2. Trunk
{noformat}
total,interval_op_rate,interval_key_rate,latency,95th,99.9th,elapsed_time
394141,39414,39414,0.0,0.0,0.0,10
1078321,68418,68418,0.0,0.0,0.0,20
1726219,64789,64789,0.0,0.0,0.0,30
2327295,60107,60107,0.0,0.0,0.0,40
2928533,60123,60123,0.0,0.0,0.0,50
3533878,60534,60534,0.0,0.0,0.0,60
3602168,6829,6829,0.0,0.0,0.0,70
3967820,36565,36565,0.0,0.0,0.0,80
4647217,67939,67939,0.0,0.0,0.0,91
5248142,60092,60092,0.0,0.0,0.0,101
5930662,68252,68252,0.0,0.0,0.0,111
6417903,48724,48724,0.0,0.0,0.0,121
6952933,53503,53503,0.0,0.0,0.0,131
7221662,26872,26872,0.0,0.0,0.0,141
7221662,0,0,0.0,0.0,0.0,151
7221662,0,0,0.0,0.0,0.0,161
7221662,0,0,0.0,0.0,0.0,172
7221662,0,0,0.0,0.0,0.0,182
7221662,0,0,0.0,0.0,0.0,192
7509240,28757,28757,0.0,0.0,0.0,202
7780984,27174,27174,0.0,0.0,0.0,212
7780984,0,0,0.0,0.0,0.0,222
7780984,0,0,0.0,0.0,0.0,232
7780984,0,0,0.0,0.0,0.0,242
8414140,63315,63315,0.0,0.0,0.0,252
8968246,55410,55410,0.0,0.0,0.0,263
9669857,70161,70161,0.0,0.0,0.0,273
10236467,56661,56661,0.0,0.0,0.0,283
10774593,53812,53812,0.0,0.0,0.0,293
10824657,5006,5006,0.0,0.0,0.0,303
11165174,34051,34051,0.0,0.0,0.0,313
11165174,0,0,0.0,0.0,0.0,323
11165174,0,0,0.0,0.0,0.0,333
11165174,0,0,0.0,0.0,0.0,343
11304248,13907,13907,0.0,0.0,0.0,354
11927380,62313,62313,0.0,0.0,0.0,364
12526960,59958,59958,0.0,0.0,0.0,374
13234647,70768,70768,0.0,0.0,0.0,384
13792652,55800,55800,0.0,0.0,0.0,394
14329718,53706,53706,0.0,0.0,0.0,404
14512350,18263,18263,0.0,0.0,0.0,414
14512929,57,57,0.0,0.0,0.0,424
14710476,19754,19754,0.0,0.0,0.0,434
14710476,0,0,0.0,0.0,0.0,445
14710476,0,0,0.0,0.0,0.0,455
15061043,35056,35056,0.0,0.0,0.0,465
15760509,69946,69946,0.0,0.0,0.0,475
16461318,70080,70080,0.0,0.0,0.0,485
17126749,66543,66543,0.0,0.0,0.0,495
17708154,58140,58140,0.0,0.0,0.0,505
18226801,51864,51864,0.0,0.0,0.0,515
18226801,0,0,0.0,0.0,0.0,526
18227225,42,42,0.0,0.0,0.0,536
18858228,63100,63100,0.0,0.0,0.0,546
19459047,60081,60081,0.0,0.0,0.0,556
19988583,52953,52953,0.0,0.0,0.0,566
2000,1141,1141,0.0,0.0,0.0,567


Averages from the middle 80% of values:
interval_op_rate  : 34003
interval_key_rate : 34003
latency median: 0.0
latency 95th percentile   : 0.0
latency 99.9th percentile : 0.0
Total operation time  : 00:09:27
END
{noformat}
h2. Patched version
{noformat}
total,interval_op_rate,interval_key_rate,latency,95th,99.9th,elapsed_time
398380,39838,39838,0.0,0.0,0.0,10
1090332,69195,69195,0.0,0.0,0.0,20
1756859,66652,66652,0.0,0.0,0.0,30
2408330,65147,65147,0.0,0.0,0.0,40
3021314,61298,61298,0.0,0.0,0.0,50
3602221,58090,58090,0.0,0.0,0.0,60
3602221,0,0,0.0,0.0,0.0,70
4086404,48418,48418,0.0,0.0,0.0,80
4670997,58459,58459,0.0,0.0,0.0,91
5328657,65766,65766,0.0,0.0,0.0,101
5950535,62187,62187,0.0,0.0,0.0,111
6544475,59394,59394,0.0,0.0,0.0,121
7163644,61916,61916,0.0,0.0,0.0,131
7307634,14399,14399,0.0,0.0,0.0,141
7331684,2405,2405,0.0,0.0,0.0,151
7989707,65802,65802,0.0,0.0,0.0,161
8653302,66359,66359,0.0,0.0,0.0,172
9273188,61988,61988,0.0,0.0,0.0,182
9935986,66279,66279,0.0,0.0,0.0,192
10489010,55302,55302,0.0,0.0,0.0,202
10909996,42098,42098,0.0,0.0,0.0,212
10962871,5287,5287,0.0,0.0,0.0,222
11274293,31142,31142,0.0,0.0,0.0,232
11274293,0,0,0.0,0.0,0.0,242
11274293,0,0,0.0,0.0,0.0,252
11297105,2281,2281,0.0,0.0,0.0,263
11946842,64973,64973,0.0,0.0,0.0,273
12509283,56244,56244,0.0,0.0,0.0,283
13205933,69665,69665,0.0,0.0,0.0,293
13809534,60360,60360,0.0,0.0,0.0,303
14334735,52520,52520,0.0,0.0,0.0,313
14615255,28052,28052,0.0,0.0,0.0,323
14615958,70,70,0.0,0.0,0.0,333
14841997,22603,22603,0.0,0.0,0.0,343
14841997,0,0,0.0,0.0,0.0,354
14841997,0,0,0.0,0.0,0.0,364
15262968,42097,42097,0.0,0.0,0.0,374
15943731,68076,68076,0.0,0.0,0.0,384
16619205,67547,67547,0.0,0.0,0.0,394
17197417,57821,57821,0.0,0.0,0.0,404
17776353,57893,57893,0.0,0.0,0.0,414
18235461,45910,45910,0.0,0.0,0.0,424
18267460,3199,3199,0.0,0.0,0.0,434
18592152,32469,32469,0.0,0.0,0.0,445
18732480,14032,14032,0.0,0.0,0.0,455
19328150,59567,59567,0.0,0.0,0.0,465

[jira] [Updated] (CASSANDRA-4338) Experiment with direct buffer in SequentialWriter

2013-10-02 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson updated CASSANDRA-4338:
---

Attachment: gc-with-patch-me.png
gc-trunk-me.png

 Experiment with direct buffer in SequentialWriter
 -

 Key: CASSANDRA-4338
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4338
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Marcus Eriksson
Priority: Minor
  Labels: performance
 Fix For: 2.1

 Attachments: 4338-gc.tar.gz, gc-4338-patched.png, gc-trunk-me.png, 
 gc-trunk.png, gc-with-patch-me.png


 Using a direct buffer instead of a heap-based byte[] should let us avoid a 
 copy into native memory when we flush the buffer.



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


[jira] [Created] (CASSANDRA-6131) JAVA_HOME on cassandra-env.sh is ignored on Debian packages

2013-10-02 Thread JIRA
Sebastián Lacuesta created CASSANDRA-6131:
-

 Summary: JAVA_HOME on cassandra-env.sh is ignored on Debian 
packages
 Key: CASSANDRA-6131
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6131
 Project: Cassandra
  Issue Type: Bug
  Components: Packaging
 Environment: I've just got upgraded to 2.0.1 package from the apache 
repositories using apt. I had the JAVA_HOME environment variable set in 
/etc/cassandra/cassandra-env.sh but after the upgrade it only worked by setting 
it on /usr/sbin/cassandra script. I can't configure java 7 system wide, only 
for cassandra.
Off-toppic: Thanks for getting rid of the jsvc mess.
Reporter: Sebastián Lacuesta






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


[jira] [Resolved] (CASSANDRA-5658) TracingStage frequently times out

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis resolved CASSANDRA-5658.
---

Resolution: Not A Problem

I'll add some code to catch the exception and explain the problem.

 TracingStage frequently times out
 -

 Key: CASSANDRA-5658
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5658
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.2.4, 1.2.6, 2.0 beta 1
Reporter: Ryan McGuire
 Attachments: 5658-logs.tar.gz, trace_bug.cql, trace_bug_cqlsh.py, 
 trace_bug.py


 I am seeing frequent timeout errors when doing programmatic traces via 
 trace_next_query()
 {code}
 ERROR [TracingStage:1] 2013-06-18 19:10:20,669 CassandraDaemon.java (line 
 196) Exception in thread Thread[TracingStage:1,5,main]
 java.lang.RuntimeException: 
 org.apache.cassandra.exceptions.WriteTimeoutException: Operation timed out - 
 received only 0 responses.
 at com.google.common.base.Throwables.propagate(Throwables.java:160)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
 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:722)
 Caused by: org.apache.cassandra.exceptions.WriteTimeoutException: Operation 
 timed out - received only 0 responses.
 at 
 org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:81)
 at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:454)
 at 
 org.apache.cassandra.tracing.TraceState$1.runMayThrow(TraceState.java:100)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 ... 3 more
 {code}
 Attached is the sample code which produced this error and the logs. The error 
 occurs directly after the INSERT statement.



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


[jira] [Updated] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes

2013-10-02 Thread Jeremiah Jordan (JIRA)

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

Jeremiah Jordan updated CASSANDRA-6127:
---

Reproduced In: 1.2.9, 1.2.6  (was: 1.2.9)

 vnodes don't scale to hundreds of nodes
 ---

 Key: CASSANDRA-6127
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Any cluster that has vnodes and consists of hundreds of 
 physical nodes.
Reporter: Tupshin Harper
Assignee: Jonathan Ellis

 There are a lot of gossip-related issues related to very wide clusters that 
 also have vnodes enabled. Let's use this ticket as a master in case there are 
 sub-tickets.
 The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge 
 instances. Each node configured with 32 vnodes.
 Without vnodes, cluster spins up fine and is ready to handle requests within 
 30 minutes or less. 
 With vnodes, nodes are reporting constant up/down flapping messages with no 
 external load on the cluster. After a couple of hours, they were still 
 flapping, had very high cpu load, and the cluster never looked like it was 
 going to stabilize or be useful for traffic.



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


[jira] [Created] (CASSANDRA-6132) CL.ANY writes can still time out

2013-10-02 Thread Jonathan Ellis (JIRA)
Jonathan Ellis created CASSANDRA-6132:
-

 Summary: CL.ANY writes can still time out
 Key: CASSANDRA-6132
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6132
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Priority: Minor


If we know that all replicas are down at the beginning of a mutation, we will 
write a hint and return success.

But if we do not, we will attemp to write to replicas, time out, return 
failure, and then write a hint, violating our contract that (unless the 
coordinator goes down), writes at CL.ANY should always succeed.



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


[jira] [Created] (CASSANDRA-6133) Tracing should deal with write failure

2013-10-02 Thread Jonathan Ellis (JIRA)
Jonathan Ellis created CASSANDRA-6133:
-

 Summary: Tracing should deal with write failure
 Key: CASSANDRA-6133
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6133
 Project: Cassandra
  Issue Type: Bug
  Components: Core, Tools
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Trivial
 Fix For: 1.2.11, 2.0.2
 Attachments: 6133.txt





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


[jira] [Updated] (CASSANDRA-6133) Tracing should deal with write failure

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-6133:
--

Attachment: 6133.txt

Attached.

 Tracing should deal with write failure
 --

 Key: CASSANDRA-6133
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6133
 Project: Cassandra
  Issue Type: Bug
  Components: Core, Tools
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Trivial
 Fix For: 1.2.11, 2.0.2

 Attachments: 6133.txt






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


[jira] [Commented] (CASSANDRA-5658) TracingStage frequently times out

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784011#comment-13784011
 ] 

Jonathan Ellis commented on CASSANDRA-5658:
---

(CASSANDRA-6132)

 TracingStage frequently times out
 -

 Key: CASSANDRA-5658
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5658
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.2.4, 1.2.6, 2.0 beta 1
Reporter: Ryan McGuire
 Attachments: 5658-logs.tar.gz, trace_bug.cql, trace_bug_cqlsh.py, 
 trace_bug.py


 I am seeing frequent timeout errors when doing programmatic traces via 
 trace_next_query()
 {code}
 ERROR [TracingStage:1] 2013-06-18 19:10:20,669 CassandraDaemon.java (line 
 196) Exception in thread Thread[TracingStage:1,5,main]
 java.lang.RuntimeException: 
 org.apache.cassandra.exceptions.WriteTimeoutException: Operation timed out - 
 received only 0 responses.
 at com.google.common.base.Throwables.propagate(Throwables.java:160)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
 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:722)
 Caused by: org.apache.cassandra.exceptions.WriteTimeoutException: Operation 
 timed out - received only 0 responses.
 at 
 org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:81)
 at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:454)
 at 
 org.apache.cassandra.tracing.TraceState$1.runMayThrow(TraceState.java:100)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 ... 3 more
 {code}
 Attached is the sample code which produced this error and the logs. The error 
 occurs directly after the INSERT statement.



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


[jira] [Comment Edited] (CASSANDRA-5658) TracingStage frequently times out

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784011#comment-13784011
 ] 

Jonathan Ellis edited comment on CASSANDRA-5658 at 10/2/13 2:36 PM:


(CASSANDRA-6133)


was (Author: jbellis):
(CASSANDRA-6132)

 TracingStage frequently times out
 -

 Key: CASSANDRA-5658
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5658
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.2.4, 1.2.6, 2.0 beta 1
Reporter: Ryan McGuire
 Attachments: 5658-logs.tar.gz, trace_bug.cql, trace_bug_cqlsh.py, 
 trace_bug.py


 I am seeing frequent timeout errors when doing programmatic traces via 
 trace_next_query()
 {code}
 ERROR [TracingStage:1] 2013-06-18 19:10:20,669 CassandraDaemon.java (line 
 196) Exception in thread Thread[TracingStage:1,5,main]
 java.lang.RuntimeException: 
 org.apache.cassandra.exceptions.WriteTimeoutException: Operation timed out - 
 received only 0 responses.
 at com.google.common.base.Throwables.propagate(Throwables.java:160)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
 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:722)
 Caused by: org.apache.cassandra.exceptions.WriteTimeoutException: Operation 
 timed out - received only 0 responses.
 at 
 org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:81)
 at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:454)
 at 
 org.apache.cassandra.tracing.TraceState$1.runMayThrow(TraceState.java:100)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 ... 3 more
 {code}
 Attached is the sample code which produced this error and the logs. The error 
 occurs directly after the INSERT statement.



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


[jira] [Assigned] (CASSANDRA-6131) JAVA_HOME on cassandra-env.sh is ignored on Debian packages

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis reassigned CASSANDRA-6131:
-

Assignee: Eric Evans

 JAVA_HOME on cassandra-env.sh is ignored on Debian packages
 ---

 Key: CASSANDRA-6131
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6131
 Project: Cassandra
  Issue Type: Bug
  Components: Packaging
 Environment: I've just got upgraded to 2.0.1 package from the apache 
 repositories using apt. I had the JAVA_HOME environment variable set in 
 /etc/cassandra/cassandra-env.sh but after the upgrade it only worked by 
 setting it on /usr/sbin/cassandra script. I can't configure java 7 system 
 wide, only for cassandra.
 Off-toppic: Thanks for getting rid of the jsvc mess.
Reporter: Sebastián Lacuesta
Assignee: Eric Evans





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


[1/3] git commit: Allow alter keyspace on system_traces

2013-10-02 Thread yukim
Updated Branches:
  refs/heads/cassandra-2.0 32c3f44a6 - 88944ffcd
  refs/heads/trunk 7a1436704 - 040e67efd


Allow alter keyspace on system_traces

patch by yukim; reviewed by jbellis for CASSANDRA-6016


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/88944ffc
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/88944ffc
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/88944ffc

Branch: refs/heads/cassandra-2.0
Commit: 88944ffcd85ae59fc92202a5a547b3c65efa03f0
Parents: 32c3f44
Author: Yuki Morishita yu...@apache.org
Authored: Wed Oct 2 10:06:53 2013 -0500
Committer: Yuki Morishita yu...@apache.org
Committed: Wed Oct 2 10:06:53 2013 -0500

--
 CHANGES.txt  |  1 +
 pylib/cqlshlib/cql3handling.py   |  2 +-
 .../org/apache/cassandra/config/DatabaseDescriptor.java  |  9 ++---
 src/java/org/apache/cassandra/config/Schema.java |  2 +-
 .../org/apache/cassandra/service/CassandraDaemon.java|  8 
 src/java/org/apache/cassandra/service/ClientState.java   | 10 +++---
 .../org/apache/cassandra/service/StorageService.java | 11 +--
 7 files changed, 25 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 80aed13..a1acc94 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
  * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
  * Delete can potentially be skipped in batch (CASSANDRA-6115)
+ * Allow alter keyspace on system_traces (CASSANDRA-6016)
 Merged from 1.2:
  * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
  * Allow estimated memtable size to exceed slab allocator size (CASSANDRA-6078)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/pylib/cqlshlib/cql3handling.py
--
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 366adcb..3b50cc9 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -43,7 +43,7 @@ class UnexpectedTableStructure(UserWarning):
 return 'Unexpected table structure; may not translate correctly to 
CQL. ' + self.msg
 
 SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth')
-NONALTERBALE_KEYSPACES = ('system', 'system_traces')
+NONALTERBALE_KEYSPACES = ('system')
 
 class Cql3ParsingRuleSet(CqlParsingRuleSet):
 keywords = set((

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 81c7308..ead9654 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -463,15 +463,10 @@ public class DatabaseDescriptor
 memtableAllocator = FBUtilities.classForName(allocatorClass, 
allocator);
 
 // Hardcoded system keyspaces
-ListKSMetaData systemKeyspaces = 
Arrays.asList(KSMetaData.systemKeyspace(), KSMetaData.traceKeyspace());
+ListKSMetaData systemKeyspaces = 
Arrays.asList(KSMetaData.systemKeyspace());
 assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
 for (KSMetaData ksmd : systemKeyspaces)
-{
-// install the definition
-for (CFMetaData cfm : ksmd.cfMetaData().values())
-Schema.instance.load(cfm);
-Schema.instance.setKeyspaceDefinition(ksmd);
-}
+Schema.instance.load(ksmd);
 
 /* Load the seeds for node contact points */
 if (conf.seed_provider == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/src/java/org/apache/cassandra/config/Schema.java
--
diff --git a/src/java/org/apache/cassandra/config/Schema.java 
b/src/java/org/apache/cassandra/config/Schema.java
index 2a9a731..d822704 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -64,7 +64,7 @@ public class Schema
 
 // 59adb24e-f3cd-3e02-97f0-5b395827453f
 public static final UUID emptyVersion;
-public static final ImmutableSetString systemKeyspaceNames = 
ImmutableSet.of(Keyspace.SYSTEM_KS, Tracing.TRACE_KS);
+public static final ImmutableSetString systemKeyspaceNames = 

[2/3] git commit: Allow alter keyspace on system_traces

2013-10-02 Thread yukim
Allow alter keyspace on system_traces

patch by yukim; reviewed by jbellis for CASSANDRA-6016


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/88944ffc
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/88944ffc
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/88944ffc

Branch: refs/heads/trunk
Commit: 88944ffcd85ae59fc92202a5a547b3c65efa03f0
Parents: 32c3f44
Author: Yuki Morishita yu...@apache.org
Authored: Wed Oct 2 10:06:53 2013 -0500
Committer: Yuki Morishita yu...@apache.org
Committed: Wed Oct 2 10:06:53 2013 -0500

--
 CHANGES.txt  |  1 +
 pylib/cqlshlib/cql3handling.py   |  2 +-
 .../org/apache/cassandra/config/DatabaseDescriptor.java  |  9 ++---
 src/java/org/apache/cassandra/config/Schema.java |  2 +-
 .../org/apache/cassandra/service/CassandraDaemon.java|  8 
 src/java/org/apache/cassandra/service/ClientState.java   | 10 +++---
 .../org/apache/cassandra/service/StorageService.java | 11 +--
 7 files changed, 25 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 80aed13..a1acc94 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
  * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
  * Delete can potentially be skipped in batch (CASSANDRA-6115)
+ * Allow alter keyspace on system_traces (CASSANDRA-6016)
 Merged from 1.2:
  * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
  * Allow estimated memtable size to exceed slab allocator size (CASSANDRA-6078)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/pylib/cqlshlib/cql3handling.py
--
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 366adcb..3b50cc9 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -43,7 +43,7 @@ class UnexpectedTableStructure(UserWarning):
 return 'Unexpected table structure; may not translate correctly to 
CQL. ' + self.msg
 
 SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth')
-NONALTERBALE_KEYSPACES = ('system', 'system_traces')
+NONALTERBALE_KEYSPACES = ('system')
 
 class Cql3ParsingRuleSet(CqlParsingRuleSet):
 keywords = set((

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 81c7308..ead9654 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -463,15 +463,10 @@ public class DatabaseDescriptor
 memtableAllocator = FBUtilities.classForName(allocatorClass, 
allocator);
 
 // Hardcoded system keyspaces
-ListKSMetaData systemKeyspaces = 
Arrays.asList(KSMetaData.systemKeyspace(), KSMetaData.traceKeyspace());
+ListKSMetaData systemKeyspaces = 
Arrays.asList(KSMetaData.systemKeyspace());
 assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
 for (KSMetaData ksmd : systemKeyspaces)
-{
-// install the definition
-for (CFMetaData cfm : ksmd.cfMetaData().values())
-Schema.instance.load(cfm);
-Schema.instance.setKeyspaceDefinition(ksmd);
-}
+Schema.instance.load(ksmd);
 
 /* Load the seeds for node contact points */
 if (conf.seed_provider == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88944ffc/src/java/org/apache/cassandra/config/Schema.java
--
diff --git a/src/java/org/apache/cassandra/config/Schema.java 
b/src/java/org/apache/cassandra/config/Schema.java
index 2a9a731..d822704 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -64,7 +64,7 @@ public class Schema
 
 // 59adb24e-f3cd-3e02-97f0-5b395827453f
 public static final UUID emptyVersion;
-public static final ImmutableSetString systemKeyspaceNames = 
ImmutableSet.of(Keyspace.SYSTEM_KS, Tracing.TRACE_KS);
+public static final ImmutableSetString systemKeyspaceNames = 
ImmutableSet.of(Keyspace.SYSTEM_KS);
 
 static
 {


[3/3] git commit: Merge branch 'cassandra-2.0' into trunk

2013-10-02 Thread yukim
Merge branch 'cassandra-2.0' into trunk

Conflicts:
src/java/org/apache/cassandra/service/CassandraDaemon.java


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/040e67ef
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/040e67ef
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/040e67ef

Branch: refs/heads/trunk
Commit: 040e67efd088dccc340d6382dfee72ad96400ffd
Parents: 7a14367 88944ff
Author: Yuki Morishita yu...@apache.org
Authored: Wed Oct 2 10:11:18 2013 -0500
Committer: Yuki Morishita yu...@apache.org
Committed: Wed Oct 2 10:11:18 2013 -0500

--
 CHANGES.txt  |  1 +
 pylib/cqlshlib/cql3handling.py   |  2 +-
 .../org/apache/cassandra/config/DatabaseDescriptor.java  |  9 ++---
 src/java/org/apache/cassandra/config/Schema.java |  2 +-
 .../org/apache/cassandra/service/CassandraDaemon.java|  8 
 src/java/org/apache/cassandra/service/ClientState.java   | 10 +++---
 .../org/apache/cassandra/service/StorageService.java | 11 +--
 7 files changed, 25 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/040e67ef/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/040e67ef/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/040e67ef/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 83bfe3a,1de5b49..536f319
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -21,10 -21,10 +21,8 @@@ import java.io.File
  import java.io.IOException;
  import java.lang.management.ManagementFactory;
  import java.net.InetAddress;
 -import java.net.MalformedURLException;
 -import java.net.URL;
  import java.util.Arrays;
  import java.util.concurrent.TimeUnit;
- import java.util.concurrent.atomic.AtomicInteger;
- 
  import javax.management.MBeanServer;
  import javax.management.ObjectName;
  import javax.management.StandardMBean;
@@@ -33,16 -33,20 +31,18 @@@ import com.addthis.metrics.reporter.con
  
  import com.google.common.collect.Iterables;
  import com.google.common.collect.SetMultimap;
- 
 -import org.apache.log4j.PropertyConfigurator;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.Schema;
- import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.db.MeteredFlusher;
+ import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.commitlog.CommitLog;
  import org.apache.cassandra.db.compaction.CompactionManager;
 -import org.apache.cassandra.db.compaction.LegacyLeveledManifest;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.io.FSError;
  import org.apache.cassandra.io.util.FileUtils;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/040e67ef/src/java/org/apache/cassandra/service/StorageService.java
--



[jira] [Assigned] (CASSANDRA-6097) nodetool repair randomly hangs.

2013-10-02 Thread Yuki Morishita (JIRA)

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

Yuki Morishita reassigned CASSANDRA-6097:
-

Assignee: Yuki Morishita

 nodetool repair randomly hangs.
 ---

 Key: CASSANDRA-6097
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6097
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: DataStax AMI
Reporter: J.B. Langston
Assignee: Yuki Morishita
Priority: Minor
 Attachments: dse.stack, nodetool.stack


 nodetool repair randomly hangs. This is not the same issue where repair hangs 
 if a stream is disrupted. This can be reproduced on a single-node cluster 
 where no streaming takes place, so I think this may be a JMX connection or 
 timeout issue. Thread dumps show that nodetool is waiting on a JMX response 
 and there are no repair-related threads running in Cassandra. Nodetool main 
 thread waiting for JMX response:
 {code}
 main prio=5 tid=7ffa4b001800 nid=0x10aedf000 in Object.wait() [10aede000]
java.lang.Thread.State: WAITING (on object monitor)
   at java.lang.Object.wait(Native Method)
   - waiting on 7f90d62e8 (a org.apache.cassandra.utils.SimpleCondition)
   at java.lang.Object.wait(Object.java:485)
   at 
 org.apache.cassandra.utils.SimpleCondition.await(SimpleCondition.java:34)
   - locked 7f90d62e8 (a org.apache.cassandra.utils.SimpleCondition)
   at 
 org.apache.cassandra.tools.RepairRunner.repairAndWait(NodeProbe.java:976)
   at 
 org.apache.cassandra.tools.NodeProbe.forceRepairAsync(NodeProbe.java:221)
   at 
 org.apache.cassandra.tools.NodeCmd.optionalKSandCFs(NodeCmd.java:1444)
   at org.apache.cassandra.tools.NodeCmd.main(NodeCmd.java:1213)
 {code}
 When nodetool hangs, it does not print out the following message:
 Starting repair command #XX, repairing 1 ranges for keyspace XXX
 However, Cassandra logs that repair in system.log:
 1380033480.95  INFO [Thread-154] 10:38:00,882 Starting repair command #X, 
 repairing X ranges for keyspace XXX
 This suggests that the repair command was received by Cassandra but the 
 connection then failed and nodetool didn't receive a response.
 Obviously, running repair on a single-node cluster is pointless but it's the 
 easiest way to demonstrate this problem. The customer who reported this has 
 also seen the issue on his real multi-node cluster.
 Steps to reproduce:
 Note: I reproduced this once on the official DataStax AMI with DSE 3.1.3 
 (Cassandra 1.2.6+patches).  I was unable to reproduce on my Mac using the 
 same version, and subsequent attempts to reproduce it on the AMI were 
 unsuccessful. The customer says he is able is able to reliably reproduce on 
 his Mac using DSE 3.1.3 and occasionally reproduce it on his real cluster. 
 1) Deploy an AMI using the DataStax AMI at 
 https://aws.amazon.com/amis/datastax-auto-clustering-ami-2-2
 2) Create a test keyspace
 {code}
 create keyspace test WITH replication = {'class': 'SimpleStrategy', 
 'replication_factor': 1};
 {code}
 3) Run an endless loop that runs nodetool repair repeatedly:
 {code}
 while true; do nodetool repair -pr test; done
 {code}
 4) Wait until repair hangs. It may take many tries; the behavior is random.



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


[jira] [Commented] (CASSANDRA-6133) Tracing should deal with write failure

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784055#comment-13784055
 ] 

Aleksey Yeschenko commented on CASSANDRA-6133:
--

Hmm. Shall we ever get UAE when writing with CL.ANY? I don't think so. Same re 
WTE, once CASSANDRA-6132 is resolved.

Only OE should be possible. I'd replace all those catch-es with a single catch 
(REE) with assert e instanseof OE and the log warning for 'too many nodes are 
overloaded to save trace events'.

Other than that +1.

 Tracing should deal with write failure
 --

 Key: CASSANDRA-6133
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6133
 Project: Cassandra
  Issue Type: Bug
  Components: Core, Tools
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Trivial
 Fix For: 1.2.11, 2.0.2

 Attachments: 6133.txt






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


[jira] [Updated] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev updated CASSANDRA-6134:
---

Attachment: BatchlogManager.txt

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Bug
Reporter: Oleg Anastasyev
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Commented] (CASSANDRA-5943) Add sstablesplit dtest

2013-10-02 Thread Daniel Meyer (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784069#comment-13784069
 ] 

Daniel Meyer commented on CASSANDRA-5943:
-

added test.  commit 24c99de22066bf7fdb4d7320d076a8dcafef397f

 Add sstablesplit dtest
 --

 Key: CASSANDRA-5943
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5943
 Project: Cassandra
  Issue Type: Test
Reporter: Brandon Williams
Assignee: Daniel Meyer
Priority: Minor

 Now that we're shipping sstablesplit, we should add a dtest to make sure it 
 works correctly.



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


[jira] [Created] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Oleg Anastasyev (JIRA)
Oleg Anastasyev created CASSANDRA-6134:
--

 Summary: More efficient BatchlogManager
 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Bug
Reporter: Oleg Anastasyev


As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.

It stores batch records in 
{code}
CREATE TABLE batchlog (
  id_partition int,
  id timeuuid,
  data blob,
  PRIMARY KEY (id_partition, id)
) WITH COMPACT STORAGE AND
  CLUSTERING ORDER BY (id DESC)
{code}

where id_partition is minute-since-epoch of id uuid. 
So when it scans for batches to replay ot scans within a single partition for  
a slice of ids since last processed date till now minus write timeout.
So no full batchlog CF scan and lot of randrom reads are made on normal cycle. 

Other improvements:
1. It runs every 1/2 of write timeout and replays all batches written within 
0.9 * write timeout from now. This way we ensure, that batched updates will be 
replayed to th moment client times out from coordinator.
2. It submits all mutations from single batch in parallel (Like StorageProxy 
do). Old implementation played them one-by-one, so client can see half applied 
batches in CF for a long time (depending on size of batch).
3. It fixes a subtle racing bug with incorrect hint ttl calculation





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


[jira] [Commented] (CASSANDRA-6117) Avoid death-by-tombstone by default

2013-10-02 Thread Oleg Anastasyev (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784077#comment-13784077
 ] 

Oleg Anastasyev commented on CASSANDRA-6117:


Guys, I see a lot of log messages like
{code}
 WARN [CompactionExecutor:158] SliceQueryFilter - Read 1 live and 1123 
tombstoned cells (see tombstone_warn_threshold)
{code}
on compaction of normal CF. 
IMHO reading a lot of thombstones on compaction and GC them is normal and 
should not be warned (nor failed with TombstoneOverwhelmingException)

 Avoid death-by-tombstone by default
 ---

 Key: CASSANDRA-6117
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6117
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
 Fix For: 2.0.2

 Attachments: 6117.txt, 6117-v2.txt, 6117-v3.txt


 We added warnings to 1.2 (CASSANDRA-6042); for 2.0 we should go farther and 
 drop requests (with an error logged) that exceed the threshold.  Users who 
 want to tread dangerously are free to crank the threshold up, but I queried 
 a lot of tombstones and Cassandra fell over is possibly the number one way 
 of killing Cassandra nodes right now.



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


[5/6] git commit: merge from 1.2

2013-10-02 Thread jbellis
merge from 1.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02d5ef3e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02d5ef3e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02d5ef3e

Branch: refs/heads/cassandra-2.0
Commit: 02d5ef3e765a34c738bc26796f2761e8cc7b715a
Parents: 88944ff 64890d8
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 10:38:08 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 10:38:39 2013 -0500

--
 CHANGES.txt |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../apache/cassandra/tracing/TraceState.java| 16 +---
 .../org/apache/cassandra/tracing/Tracing.java   | 40 
 4 files changed, 45 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/CHANGES.txt
--
diff --cc CHANGES.txt
index a1acc94,5267709..f012ed1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,21 -1,5 +1,22 @@@
 -1.2.11
 +2.0.2
 + * Add configurable metrics reporting (CASSANDRA-4430)
 + * drop queries exceeding a configurable number of tombstones (CASSANDRA-6117)
 + * Track and persist sstable read activity (CASSANDRA-5515)
 + * Fixes for speculative retry (CASSANDRA-5932)
 + * Improve memory usage of metadata min/max column names (CASSANDRA-6077)
 + * Fix thrift validation refusing row markers on CQL3 tables (CASSANDRA-6081)
 + * Fix insertion of collections with CAS (CASSANDRA-6069)
 + * Correctly send metadata on SELECT COUNT (CASSANDRA-6080)
 + * Track clients' remote addresses in ClientState (CASSANDRA-6070)
 + * Create snapshot dir if it does not exist when migrating
 +   leveled manifest (CASSANDRA-6093)
 + * make sequential nodetool repair the default (CASSANDRA-5950)
 + * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
 + * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
 + * Delete can potentially be skipped in batch (CASSANDRA-6115)
 + * Allow alter keyspace on system_traces (CASSANDRA-6016)
 +Merged from 1.2:
+  * Tracing should log write failure rather than raw exceptions 
(CASSANDRA-6133)
   * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
   * Allow estimated memtable size to exceed slab allocator size 
(CASSANDRA-6078)
   * Start MeteredFlusher earlier to prevent OOM during CL replay 
(CASSANDRA-6087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/src/java/org/apache/cassandra/config/KSMetaData.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/src/java/org/apache/cassandra/tracing/TraceState.java
--
diff --cc src/java/org/apache/cassandra/tracing/TraceState.java
index 82bda2d,326b351..b7d67a4
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@@ -29,9 -28,12 +28,15 @@@ import org.slf4j.helpers.MessageFormatt
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;
  import org.apache.cassandra.config.CFMetaData;
- import org.apache.cassandra.db.*;
++import org.apache.cassandra.db.ArrayBackedSortedColumns;
+ import org.apache.cassandra.db.ColumnFamily;
++import org.apache.cassandra.db.ConsistencyLevel;
+ import org.apache.cassandra.db.RowMutation;
 +import org.apache.cassandra.service.StorageProxy;
- import org.apache.cassandra.utils.*;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.FBUtilities;
+ import org.apache.cassandra.utils.UUIDGen;
+ import org.apache.cassandra.utils.WrappedRunnable;
  
  /**
   * ThreadLocal state for a tracing session. The presence of an instance of 
this class as a ThreadLocal denotes that an
@@@ -89,12 -91,12 +94,12 @@@ public class TraceStat
  
  StageManager.getStage(Stage.TRACING).execute(new WrappedRunnable()
  {
- public void runMayThrow() throws Exception
+ public void runMayThrow()
  {
  CFMetaData cfMeta = CFMetaData.TraceEventsCf;
 -ColumnFamily cf = ColumnFamily.create(cfMeta);
 +ColumnFamily cf = 
ArrayBackedSortedColumns.factory.create(cfMeta);
 +Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(activity)), message);
  Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(source)), FBUtilities.getBroadcastAddress());
 -Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(thread)), threadName);
  

[6/6] git commit: Merge branch 'cassandra-2.0' into trunk

2013-10-02 Thread jbellis
Merge branch 'cassandra-2.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8bdba79f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8bdba79f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8bdba79f

Branch: refs/heads/trunk
Commit: 8bdba79f978a56dd5b717d271ced5c0f0cf8
Parents: 040e67e 02d5ef3
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 10:38:57 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 10:38:57 2013 -0500

--
 CHANGES.txt |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../apache/cassandra/tracing/TraceState.java| 16 +---
 .../org/apache/cassandra/tracing/Tracing.java   | 40 
 4 files changed, 45 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bdba79f/CHANGES.txt
--



[4/6] git commit: merge from 1.2

2013-10-02 Thread jbellis
merge from 1.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02d5ef3e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02d5ef3e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02d5ef3e

Branch: refs/heads/trunk
Commit: 02d5ef3e765a34c738bc26796f2761e8cc7b715a
Parents: 88944ff 64890d8
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 10:38:08 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 10:38:39 2013 -0500

--
 CHANGES.txt |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../apache/cassandra/tracing/TraceState.java| 16 +---
 .../org/apache/cassandra/tracing/Tracing.java   | 40 
 4 files changed, 45 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/CHANGES.txt
--
diff --cc CHANGES.txt
index a1acc94,5267709..f012ed1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,21 -1,5 +1,22 @@@
 -1.2.11
 +2.0.2
 + * Add configurable metrics reporting (CASSANDRA-4430)
 + * drop queries exceeding a configurable number of tombstones (CASSANDRA-6117)
 + * Track and persist sstable read activity (CASSANDRA-5515)
 + * Fixes for speculative retry (CASSANDRA-5932)
 + * Improve memory usage of metadata min/max column names (CASSANDRA-6077)
 + * Fix thrift validation refusing row markers on CQL3 tables (CASSANDRA-6081)
 + * Fix insertion of collections with CAS (CASSANDRA-6069)
 + * Correctly send metadata on SELECT COUNT (CASSANDRA-6080)
 + * Track clients' remote addresses in ClientState (CASSANDRA-6070)
 + * Create snapshot dir if it does not exist when migrating
 +   leveled manifest (CASSANDRA-6093)
 + * make sequential nodetool repair the default (CASSANDRA-5950)
 + * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
 + * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
 + * Delete can potentially be skipped in batch (CASSANDRA-6115)
 + * Allow alter keyspace on system_traces (CASSANDRA-6016)
 +Merged from 1.2:
+  * Tracing should log write failure rather than raw exceptions 
(CASSANDRA-6133)
   * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
   * Allow estimated memtable size to exceed slab allocator size 
(CASSANDRA-6078)
   * Start MeteredFlusher earlier to prevent OOM during CL replay 
(CASSANDRA-6087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/src/java/org/apache/cassandra/config/KSMetaData.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/02d5ef3e/src/java/org/apache/cassandra/tracing/TraceState.java
--
diff --cc src/java/org/apache/cassandra/tracing/TraceState.java
index 82bda2d,326b351..b7d67a4
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@@ -29,9 -28,12 +28,15 @@@ import org.slf4j.helpers.MessageFormatt
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;
  import org.apache.cassandra.config.CFMetaData;
- import org.apache.cassandra.db.*;
++import org.apache.cassandra.db.ArrayBackedSortedColumns;
+ import org.apache.cassandra.db.ColumnFamily;
++import org.apache.cassandra.db.ConsistencyLevel;
+ import org.apache.cassandra.db.RowMutation;
 +import org.apache.cassandra.service.StorageProxy;
- import org.apache.cassandra.utils.*;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.FBUtilities;
+ import org.apache.cassandra.utils.UUIDGen;
+ import org.apache.cassandra.utils.WrappedRunnable;
  
  /**
   * ThreadLocal state for a tracing session. The presence of an instance of 
this class as a ThreadLocal denotes that an
@@@ -89,12 -91,12 +94,12 @@@ public class TraceStat
  
  StageManager.getStage(Stage.TRACING).execute(new WrappedRunnable()
  {
- public void runMayThrow() throws Exception
+ public void runMayThrow()
  {
  CFMetaData cfMeta = CFMetaData.TraceEventsCf;
 -ColumnFamily cf = ColumnFamily.create(cfMeta);
 +ColumnFamily cf = 
ArrayBackedSortedColumns.factory.create(cfMeta);
 +Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(activity)), message);
  Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(source)), FBUtilities.getBroadcastAddress());
 -Tracing.addColumn(cf, Tracing.buildName(cfMeta, eventId, 
ByteBufferUtil.bytes(thread)), threadName);
  if 

[1/6] git commit: Tracing should log write failure rather than raw exceptions patch by jbellis; reviewed by ayeschenko for CASSANDRA-6133

2013-10-02 Thread jbellis
Updated Branches:
  refs/heads/cassandra-1.2 672131d97 - 64890d86d
  refs/heads/cassandra-2.0 88944ffcd - 02d5ef3e7
  refs/heads/trunk 040e67efd - 8bdba79f9


Tracing should log write failure rather than raw exceptions
patch by jbellis; reviewed by ayeschenko for CASSANDRA-6133


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/64890d86
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/64890d86
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/64890d86

Branch: refs/heads/cassandra-1.2
Commit: 64890d86d87ec527b2f7fac2bc4d80712c290268
Parents: 672131d
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 10:35:26 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 10:35:26 2013 -0500

--
 CHANGES.txt |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../org/apache/cassandra/db/RowMutation.java|  5 +++
 .../apache/cassandra/tracing/TraceState.java| 16 
 .../org/apache/cassandra/tracing/Tracing.java   | 43 ++--
 5 files changed, 45 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 6a695e1..5267709 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.2.11
+ * Tracing should log write failure rather than raw exceptions (CASSANDRA-6133)
  * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
  * Allow estimated memtable size to exceed slab allocator size (CASSANDRA-6078)
  * Start MeteredFlusher earlier to prevent OOM during CL replay 
(CASSANDRA-6087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/config/KSMetaData.java
--
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java 
b/src/java/org/apache/cassandra/config/KSMetaData.java
index b92f9a5..79dff35 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -99,7 +99,7 @@ public final class KSMetaData
 public static KSMetaData traceKeyspace()
 {
 ListCFMetaData cfDefs = Arrays.asList(CFMetaData.TraceSessionsCf, 
CFMetaData.TraceEventsCf);
-return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, 
ImmutableMap.of(replication_factor, 1), true, cfDefs);
+return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, 
ImmutableMap.of(replication_factor, 2), true, cfDefs);
 }
 
 public static KSMetaData testMetadata(String name, Class? extends 
AbstractReplicationStrategy strategyClass, MapString, String 
strategyOptions, CFMetaData... cfDefs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/db/RowMutation.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java 
b/src/java/org/apache/cassandra/db/RowMutation.java
index 69f4a5f..5000d79 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -56,6 +56,11 @@ public class RowMutation implements IMutation
 this(table, key, new HashMapUUID, ColumnFamily());
 }
 
+public RowMutation(String keyspaceName, ByteBuffer key, ColumnFamily cf)
+{
+this(keyspaceName, key, Collections.singletonMap(cf.id(), cf));
+}
+
 public RowMutation(String table, Row row)
 {
 this(table, row.key.key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/tracing/TraceState.java
--
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java 
b/src/java/org/apache/cassandra/tracing/TraceState.java
index 4d52f8f..326b351 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.tracing;
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
@@ -30,10 +29,11 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import 

[2/6] git commit: Tracing should log write failure rather than raw exceptions patch by jbellis; reviewed by ayeschenko for CASSANDRA-6133

2013-10-02 Thread jbellis
Tracing should log write failure rather than raw exceptions
patch by jbellis; reviewed by ayeschenko for CASSANDRA-6133


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/64890d86
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/64890d86
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/64890d86

Branch: refs/heads/cassandra-2.0
Commit: 64890d86d87ec527b2f7fac2bc4d80712c290268
Parents: 672131d
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 10:35:26 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 10:35:26 2013 -0500

--
 CHANGES.txt |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../org/apache/cassandra/db/RowMutation.java|  5 +++
 .../apache/cassandra/tracing/TraceState.java| 16 
 .../org/apache/cassandra/tracing/Tracing.java   | 43 ++--
 5 files changed, 45 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 6a695e1..5267709 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.2.11
+ * Tracing should log write failure rather than raw exceptions (CASSANDRA-6133)
  * lock access to TM.endpointToHostIdMap (CASSANDRA-6103)
  * Allow estimated memtable size to exceed slab allocator size (CASSANDRA-6078)
  * Start MeteredFlusher earlier to prevent OOM during CL replay 
(CASSANDRA-6087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/config/KSMetaData.java
--
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java 
b/src/java/org/apache/cassandra/config/KSMetaData.java
index b92f9a5..79dff35 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -99,7 +99,7 @@ public final class KSMetaData
 public static KSMetaData traceKeyspace()
 {
 ListCFMetaData cfDefs = Arrays.asList(CFMetaData.TraceSessionsCf, 
CFMetaData.TraceEventsCf);
-return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, 
ImmutableMap.of(replication_factor, 1), true, cfDefs);
+return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, 
ImmutableMap.of(replication_factor, 2), true, cfDefs);
 }
 
 public static KSMetaData testMetadata(String name, Class? extends 
AbstractReplicationStrategy strategyClass, MapString, String 
strategyOptions, CFMetaData... cfDefs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/db/RowMutation.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java 
b/src/java/org/apache/cassandra/db/RowMutation.java
index 69f4a5f..5000d79 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -56,6 +56,11 @@ public class RowMutation implements IMutation
 this(table, key, new HashMapUUID, ColumnFamily());
 }
 
+public RowMutation(String keyspaceName, ByteBuffer key, ColumnFamily cf)
+{
+this(keyspaceName, key, Collections.singletonMap(cf.id(), cf));
+}
+
 public RowMutation(String table, Row row)
 {
 this(table, row.key.key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64890d86/src/java/org/apache/cassandra/tracing/TraceState.java
--
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java 
b/src/java/org/apache/cassandra/tracing/TraceState.java
index 4d52f8f..326b351 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.tracing;
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
@@ -30,10 +29,11 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.WrappedRunnable;
 
 /**
  * ThreadLocal state for a tracing session. 

[jira] [Commented] (CASSANDRA-5932) Speculative read performance data show unexpected results

2013-10-02 Thread Li Zou (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784089#comment-13784089
 ] 

Li Zou commented on CASSANDRA-5932:
---

This testing result is reasonable and what is expected.

For PERCENTILE / CUSTOM configuration, the larger the 
{{cfs.sampleLatencyNanos}} the smaller the throughput impact for normal 
operations before the outage. However, during the outage period, the situation 
is reversed, i.e. the smaller {{cfs.sampleLatencyNanos}}, the smaller the 
throughput impact will be, as it times out quicker and triggers the speculative 
retries.

For the ALWAYS configuration, as it always sends out one speculative in 
addition to the usual read requests, the throughput performance should be lower 
than those of PERCENTILE / CUSTOM for normal operations before the outage. 
Since it always sends out the speculative retries, the throughput impact during 
the outage period should be the smallest. The testing result indicates that 
this is true.


 Speculative read performance data show unexpected results
 -

 Key: CASSANDRA-5932
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5932
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan McGuire
Assignee: Aleksey Yeschenko
 Fix For: 2.0.2

 Attachments: 5932.6692c50412ef7d.compaction.png, 
 5932-6692c50412ef7d.png, 5932.6692c50412ef7d.rr0.png, 
 5932.ded39c7e1c2fa.logs.tar.gz, 5932.txt, 5933-128_and_200rc1.png, 
 5933-7a87fc11.png, 5933-logs.tar.gz, 5933-randomized-dsnitch-replica.2.png, 
 5933-randomized-dsnitch-replica.3.png, 5933-randomized-dsnitch-replica.png, 
 compaction-makes-slow.png, compaction-makes-slow-stats.png, 
 eager-read-looks-promising.png, eager-read-looks-promising-stats.png, 
 eager-read-not-consistent.png, eager-read-not-consistent-stats.png, 
 node-down-increase-performance.png


 I've done a series of stress tests with eager retries enabled that show 
 undesirable behavior. I'm grouping these behaviours into one ticket as they 
 are most likely related.
 1) Killing off a node in a 4 node cluster actually increases performance.
 2) Compactions make nodes slow, even after the compaction is done.
 3) Eager Reads tend to lessen the *immediate* performance impact of a node 
 going down, but not consistently.
 My Environment:
 1 stress machine: node0
 4 C* nodes: node4, node5, node6, node7
 My script:
 node0 writes some data: stress -d node4 -F 3000 -n 3000 -i 5 -l 2 -K 
 20
 node0 reads some data: stress -d node4 -n 3000 -o read -i 5 -K 20
 h3. Examples:
 h5. A node going down increases performance:
 !node-down-increase-performance.png!
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 At 450s, I kill -9 one of the nodes. There is a brief decrease in performance 
 as the snitch adapts, but then it recovers... to even higher performance than 
 before.
 h5. Compactions make nodes permanently slow:
 !compaction-makes-slow.png!
 !compaction-makes-slow-stats.png!
 The green and orange lines represent trials with eager retry enabled, they 
 never recover their op-rate from before the compaction as the red and blue 
 lines do.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.compaction.2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. Speculative Read tends to lessen the *immediate* impact:
 !eager-read-looks-promising.png!
 !eager-read-looks-promising-stats.png!
 This graph looked the most promising to me, the two trials with eager retry, 
 the green and orange line, at 450s showed the smallest dip in performance. 
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. But not always:
 !eager-read-not-consistent.png!
 !eager-read-not-consistent-stats.png!
 This is a retrial with the same settings as above, yet the 95percentile eager 
 retry (red line) did poorly this time at 450s.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.rc1.try2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]



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


[jira] [Commented] (CASSANDRA-5932) Speculative read performance data show unexpected results

2013-10-02 Thread Ryan McGuire (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784094#comment-13784094
 ] 

Ryan McGuire commented on CASSANDRA-5932:
-

With read_repair_chance = 1

!5932.6692c50412ef7d.rr1.png!

[data 
here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.5933.6692c50412ef7d.node_killed.rr1.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]

 Speculative read performance data show unexpected results
 -

 Key: CASSANDRA-5932
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5932
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan McGuire
Assignee: Aleksey Yeschenko
 Fix For: 2.0.2

 Attachments: 5932.6692c50412ef7d.compaction.png, 
 5932-6692c50412ef7d.png, 5932.6692c50412ef7d.rr0.png, 
 5932.6692c50412ef7d.rr1.png, 5932.ded39c7e1c2fa.logs.tar.gz, 5932.txt, 
 5933-128_and_200rc1.png, 5933-7a87fc11.png, 5933-logs.tar.gz, 
 5933-randomized-dsnitch-replica.2.png, 5933-randomized-dsnitch-replica.3.png, 
 5933-randomized-dsnitch-replica.png, compaction-makes-slow.png, 
 compaction-makes-slow-stats.png, eager-read-looks-promising.png, 
 eager-read-looks-promising-stats.png, eager-read-not-consistent.png, 
 eager-read-not-consistent-stats.png, node-down-increase-performance.png


 I've done a series of stress tests with eager retries enabled that show 
 undesirable behavior. I'm grouping these behaviours into one ticket as they 
 are most likely related.
 1) Killing off a node in a 4 node cluster actually increases performance.
 2) Compactions make nodes slow, even after the compaction is done.
 3) Eager Reads tend to lessen the *immediate* performance impact of a node 
 going down, but not consistently.
 My Environment:
 1 stress machine: node0
 4 C* nodes: node4, node5, node6, node7
 My script:
 node0 writes some data: stress -d node4 -F 3000 -n 3000 -i 5 -l 2 -K 
 20
 node0 reads some data: stress -d node4 -n 3000 -o read -i 5 -K 20
 h3. Examples:
 h5. A node going down increases performance:
 !node-down-increase-performance.png!
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 At 450s, I kill -9 one of the nodes. There is a brief decrease in performance 
 as the snitch adapts, but then it recovers... to even higher performance than 
 before.
 h5. Compactions make nodes permanently slow:
 !compaction-makes-slow.png!
 !compaction-makes-slow-stats.png!
 The green and orange lines represent trials with eager retry enabled, they 
 never recover their op-rate from before the compaction as the red and blue 
 lines do.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.compaction.2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. Speculative Read tends to lessen the *immediate* impact:
 !eager-read-looks-promising.png!
 !eager-read-looks-promising-stats.png!
 This graph looked the most promising to me, the two trials with eager retry, 
 the green and orange line, at 450s showed the smallest dip in performance. 
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. But not always:
 !eager-read-not-consistent.png!
 !eager-read-not-consistent-stats.png!
 This is a retrial with the same settings as above, yet the 95percentile eager 
 retry (red line) did poorly this time at 450s.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.rc1.try2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]



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


[jira] [Updated] (CASSANDRA-5932) Speculative read performance data show unexpected results

2013-10-02 Thread Ryan McGuire (JIRA)

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

Ryan McGuire updated CASSANDRA-5932:


Attachment: 5932.6692c50412ef7d.rr1.png

 Speculative read performance data show unexpected results
 -

 Key: CASSANDRA-5932
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5932
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan McGuire
Assignee: Aleksey Yeschenko
 Fix For: 2.0.2

 Attachments: 5932.6692c50412ef7d.compaction.png, 
 5932-6692c50412ef7d.png, 5932.6692c50412ef7d.rr0.png, 
 5932.6692c50412ef7d.rr1.png, 5932.ded39c7e1c2fa.logs.tar.gz, 5932.txt, 
 5933-128_and_200rc1.png, 5933-7a87fc11.png, 5933-logs.tar.gz, 
 5933-randomized-dsnitch-replica.2.png, 5933-randomized-dsnitch-replica.3.png, 
 5933-randomized-dsnitch-replica.png, compaction-makes-slow.png, 
 compaction-makes-slow-stats.png, eager-read-looks-promising.png, 
 eager-read-looks-promising-stats.png, eager-read-not-consistent.png, 
 eager-read-not-consistent-stats.png, node-down-increase-performance.png


 I've done a series of stress tests with eager retries enabled that show 
 undesirable behavior. I'm grouping these behaviours into one ticket as they 
 are most likely related.
 1) Killing off a node in a 4 node cluster actually increases performance.
 2) Compactions make nodes slow, even after the compaction is done.
 3) Eager Reads tend to lessen the *immediate* performance impact of a node 
 going down, but not consistently.
 My Environment:
 1 stress machine: node0
 4 C* nodes: node4, node5, node6, node7
 My script:
 node0 writes some data: stress -d node4 -F 3000 -n 3000 -i 5 -l 2 -K 
 20
 node0 reads some data: stress -d node4 -n 3000 -o read -i 5 -K 20
 h3. Examples:
 h5. A node going down increases performance:
 !node-down-increase-performance.png!
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 At 450s, I kill -9 one of the nodes. There is a brief decrease in performance 
 as the snitch adapts, but then it recovers... to even higher performance than 
 before.
 h5. Compactions make nodes permanently slow:
 !compaction-makes-slow.png!
 !compaction-makes-slow-stats.png!
 The green and orange lines represent trials with eager retry enabled, they 
 never recover their op-rate from before the compaction as the red and blue 
 lines do.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.compaction.2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. Speculative Read tends to lessen the *immediate* impact:
 !eager-read-looks-promising.png!
 !eager-read-looks-promising-stats.png!
 This graph looked the most promising to me, the two trials with eager retry, 
 the green and orange line, at 450s showed the smallest dip in performance. 
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]
 h5. But not always:
 !eager-read-not-consistent.png!
 !eager-read-not-consistent-stats.png!
 This is a retrial with the same settings as above, yet the 95percentile eager 
 retry (red line) did poorly this time at 450s.
 [Data for this test 
 here|http://ryanmcguire.info/ds/graph/graph.html?stats=stats.eager_retry.node_killed.just_20.rc1.try2.jsonmetric=interval_op_rateoperation=stress-readsmoothing=1]



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


[jira] [Updated] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko updated CASSANDRA-6134:
-

  Reviewer: Aleksey Yeschenko
  Priority: Minor  (was: Major)
Issue Type: Improvement  (was: Bug)

Haven't looked at it carefully yet, but, you can't change the batchlog cf like 
this and breaking backward compatibility entirely.

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Comment Edited] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784099#comment-13784099
 ] 

Aleksey Yeschenko edited comment on CASSANDRA-6134 at 10/2/13 3:53 PM:
---

Haven't looked at it carefully yet, but, you can't change the batchlog cf like 
this and break backward compatibility entirely.


was (Author: iamaleksey):
Haven't looked at it carefully yet, but, you can't change the batchlog cf like 
this and breaking backward compatibility entirely.

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Commented] (CASSANDRA-6117) Avoid death-by-tombstone by default

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784108#comment-13784108
 ] 

Jonathan Ellis commented on CASSANDRA-6117:
---

Was already excepting compaction for fail threshold; added exception to warn as 
well in 546cbf5ffc8205e361ed9cf57b44faa384097c45

 Avoid death-by-tombstone by default
 ---

 Key: CASSANDRA-6117
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6117
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
 Fix For: 2.0.2

 Attachments: 6117.txt, 6117-v2.txt, 6117-v3.txt


 We added warnings to 1.2 (CASSANDRA-6042); for 2.0 we should go farther and 
 drop requests (with an error logged) that exceed the threshold.  Users who 
 want to tread dangerously are free to crank the threshold up, but I queried 
 a lot of tombstones and Cassandra fell over is possibly the number one way 
 of killing Cassandra nodes right now.



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


[1/3] git commit: avoid logging tombstone warnings during compaction

2013-10-02 Thread jbellis
Updated Branches:
  refs/heads/cassandra-2.0 02d5ef3e7 - 546cbf5ff
  refs/heads/trunk 8bdba79f9 - 7dd9ea2e8


avoid logging tombstone warnings during compaction


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/546cbf5f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/546cbf5f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/546cbf5f

Branch: refs/heads/cassandra-2.0
Commit: 546cbf5ffc8205e361ed9cf57b44faa384097c45
Parents: 02d5ef3
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 11:00:04 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 11:00:04 2013 -0500

--
 .../cassandra/db/columniterator/IdentityQueryFilter.java   | 2 +-
 src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java  | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/546cbf5f/src/java/org/apache/cassandra/db/columniterator/IdentityQueryFilter.java
--
diff --git 
a/src/java/org/apache/cassandra/db/columniterator/IdentityQueryFilter.java 
b/src/java/org/apache/cassandra/db/columniterator/IdentityQueryFilter.java
index 7b1085a..828f7e5 100644
--- a/src/java/org/apache/cassandra/db/columniterator/IdentityQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/columniterator/IdentityQueryFilter.java
@@ -31,7 +31,7 @@ public class IdentityQueryFilter extends SliceQueryFilter
 }
 
 @Override
-protected boolean respectTombstoneFailures()
+protected boolean respectTombstoneThresholds()
 {
 return false;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/546cbf5f/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
--
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java 
b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index 9c67d71..794d31c 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -194,7 +194,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
 if (columnCounter.live()  count)
 break;
 
-if (respectTombstoneFailures()  columnCounter.ignored()  
DatabaseDescriptor.getTombstoneFailureThreshold())
+if (respectTombstoneThresholds()  columnCounter.ignored()  
DatabaseDescriptor.getTombstoneFailureThreshold())
 {
 Tracing.trace(Scanned over {} tombstones; query aborted (see 
tombstone_fail_threshold), DatabaseDescriptor.getTombstoneFailureThreshold());
 logger.error(Scanned over {} tombstones; query aborted (see 
tombstone_fail_threshold), DatabaseDescriptor.getTombstoneFailureThreshold());
@@ -205,11 +205,11 @@ public class SliceQueryFilter implements IDiskAtomFilter
 }
 
 Tracing.trace(Read {} live and {} tombstoned cells, 
columnCounter.live(), columnCounter.ignored());
-if (columnCounter.ignored()  
DatabaseDescriptor.getTombstoneWarnThreshold())
+if (respectTombstoneThresholds()  columnCounter.ignored()  
DatabaseDescriptor.getTombstoneWarnThreshold())
 logger.warn(Read {} live and {} tombstoned cells (see 
tombstone_warn_threshold), columnCounter.live(), columnCounter.ignored());
 }
 
-protected boolean respectTombstoneFailures()
+protected boolean respectTombstoneThresholds()
 {
 return true;
 }



[3/3] git commit: Merge branch 'cassandra-2.0' into trunk

2013-10-02 Thread jbellis
Merge branch 'cassandra-2.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7dd9ea2e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7dd9ea2e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7dd9ea2e

Branch: refs/heads/trunk
Commit: 7dd9ea2e864127448857dfdf9cca021a86f9072f
Parents: 8bdba79 546cbf5
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 11:01:49 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 11:01:49 2013 -0500

--
 .../cassandra/db/columniterator/IdentityQueryFilter.java   | 2 +-
 src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java  | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
--




[jira] [Commented] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Oleg Anastasyev (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784111#comment-13784111
 ] 

Oleg Anastasyev commented on CASSANDRA-6134:


Well, the way how to migrate old batchlog records is a subject to discussion 
and TBD. The easiest way is to have batchlog2 CF with new definition and 
batchlog with old one. But i find it somewhat ugly.

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Commented] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784115#comment-13784115
 ] 

Jonathan Ellis commented on CASSANDRA-6134:
---

I vote for easy in this case, if users get so low level that they care what 
this table is named then they have no right to be offended. :)

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Commented] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784122#comment-13784122
 ] 

Aleksey Yeschenko commented on CASSANDRA-6134:
--

If the changes are worth it in practice *AND* if there is absolutely no way to 
reuse the current schema, you still have to migrate the old batches.

 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Commented] (CASSANDRA-6134) More efficient BatchlogManager

2013-10-02 Thread Oleg Anastasyev (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784148#comment-13784148
 ] 

Oleg Anastasyev commented on CASSANDRA-6134:


Alex: It seems that current schema completely incompatible with new one. 
So, could you plz then look and decide is new batchlog manager useful for you, 
so it is worth to implement migration.




 More efficient BatchlogManager
 --

 Key: CASSANDRA-6134
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6134
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oleg Anastasyev
Priority: Minor
 Attachments: BatchlogManager.txt


 As we discussed earlier in CASSANDRA-6079 this is the new BatchManager.
 It stores batch records in 
 {code}
 CREATE TABLE batchlog (
   id_partition int,
   id timeuuid,
   data blob,
   PRIMARY KEY (id_partition, id)
 ) WITH COMPACT STORAGE AND
   CLUSTERING ORDER BY (id DESC)
 {code}
 where id_partition is minute-since-epoch of id uuid. 
 So when it scans for batches to replay ot scans within a single partition for 
  a slice of ids since last processed date till now minus write timeout.
 So no full batchlog CF scan and lot of randrom reads are made on normal 
 cycle. 
 Other improvements:
 1. It runs every 1/2 of write timeout and replays all batches written within 
 0.9 * write timeout from now. This way we ensure, that batched updates will 
 be replayed to th moment client times out from coordinator.
 2. It submits all mutations from single batch in parallel (Like StorageProxy 
 do). Old implementation played them one-by-one, so client can see half 
 applied batches in CF for a long time (depending on size of batch).
 3. It fixes a subtle racing bug with incorrect hint ttl calculation



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


[jira] [Updated] (CASSANDRA-6132) CL.ANY writes can still time out

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-6132:
--

Attachment: 6132.txt

 CL.ANY writes can still time out
 

 Key: CASSANDRA-6132
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6132
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Priority: Minor
 Fix For: 1.2.11, 2.0.2

 Attachments: 6132.txt


 If we know that all replicas are down at the beginning of a mutation, we will 
 write a hint and return success.
 But if we do not, we will attemp to write to replicas, time out, return 
 failure, and then write a hint, violating our contract that (unless the 
 coordinator goes down), writes at CL.ANY should always succeed.



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


[jira] [Commented] (CASSANDRA-5970) FilteredRangeSlice command for regex searches against column names on known sets of keys

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784177#comment-13784177
 ] 

Jonathan Ellis commented on CASSANDRA-5970:
---

How is that looking so far?

 FilteredRangeSlice command for regex searches against column names on known 
 sets of keys
 

 Key: CASSANDRA-5970
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5970
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Nate McCall

 This is the ability to apply a regex against columns when the set of keys is 
 known. In filtering the keys, we would like to allow for the following 
 clauses: E, GTE, LTE, NE, inclusive list, exclusive list.
 The end goal is to provide for efficient searching in the case where you have 
 some knowledge of the keys. A specific use case would be, say, searching user 
 agent strings in the given set of date buckets in the classic time-series web 
 log use case. This is a sweet spot for Cassandra and providing a more 
 direct method of access for such will help a lot of users.
 Additionally, this will provide some level of feature parity with RDBMS crowd 
 who've had this feature for some time.
 Internally, this will include the introduction of a new Verb, SSTableScanner 
 extension and an ExtendedFilter implementation which applies the regex as 
 well as a new method on StorageProxy.
 This issue does not cover exposing this new query method to thrift and CQL, 
 but obviously that will be required for this to be of any practical use. 
 Those should be covered by separate issues.



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


[jira] [Assigned] (CASSANDRA-5970) FilteredRangeSlice command for regex searches against column names on known sets of keys

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis reassigned CASSANDRA-5970:
-

Assignee: Nate McCall

 FilteredRangeSlice command for regex searches against column names on known 
 sets of keys
 

 Key: CASSANDRA-5970
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5970
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Nate McCall
Assignee: Nate McCall

 This is the ability to apply a regex against columns when the set of keys is 
 known. In filtering the keys, we would like to allow for the following 
 clauses: E, GTE, LTE, NE, inclusive list, exclusive list.
 The end goal is to provide for efficient searching in the case where you have 
 some knowledge of the keys. A specific use case would be, say, searching user 
 agent strings in the given set of date buckets in the classic time-series web 
 log use case. This is a sweet spot for Cassandra and providing a more 
 direct method of access for such will help a lot of users.
 Additionally, this will provide some level of feature parity with RDBMS crowd 
 who've had this feature for some time.
 Internally, this will include the introduction of a new Verb, SSTableScanner 
 extension and an ExtendedFilter implementation which applies the regex as 
 well as a new method on StorageProxy.
 This issue does not cover exposing this new query method to thrift and CQL, 
 but obviously that will be required for this to be of any practical use. 
 Those should be covered by separate issues.



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


[jira] [Assigned] (CASSANDRA-4549) Update the pig examples to include more recent pig/cassandra features

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis reassigned CASSANDRA-4549:
-

Assignee: Alex Liu  (was: Jeremy Hanna)

 Update the pig examples to include more recent pig/cassandra features
 -

 Key: CASSANDRA-4549
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4549
 Project: Cassandra
  Issue Type: Task
  Components: Hadoop
Reporter: Jeremy Hanna
Assignee: Alex Liu
Priority: Minor

 Now that there is support for a variety of Cassandra features from Pig (esp 
 1.1+), it would great to have some of them in the examples so that people can 
 see how to use them.



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


[jira] [Commented] (CASSANDRA-4757) Expose bulk loading progress/status over jmx

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4757?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784180#comment-13784180
 ] 

Jonathan Ellis commented on CASSANDRA-4757:
---

[~gdeangel], are you still working on this?

 Expose bulk loading progress/status over jmx
 

 Key: CASSANDRA-4757
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4757
 Project: Cassandra
  Issue Type: Improvement
Reporter: Nick Bailey
Assignee: Greg DeAngelis
Priority: Minor
  Labels: lhf
 Fix For: 2.0.0

 Attachments: CASSANDRA-4757.txt, CASSANDRA-4757v2.txt


 The bulk loading interface should be exposing some progress or status 
 information over jmx. This shouldn't be too difficult and should be exposed 
 in a way that the information is available whether you are using the separate 
 sstableloader utility or calling the bulkload jmx call.



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


[jira] [Commented] (CASSANDRA-5433) Nodetool - Human Readable

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5433?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784182#comment-13784182
 ] 

Jonathan Ellis commented on CASSANDRA-5433:
---

[~gdeangel], are you still working on this?

 Nodetool - Human Readable
 -

 Key: CASSANDRA-5433
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5433
 Project: Cassandra
  Issue Type: Improvement
Reporter: Brooke Bryan
Assignee: Greg DeAngelis
Priority: Trivial
 Attachments: humanreadablecompactionstats.patch


 Would be great to have a human readable option in nodetool to easily look 
 stats without having to convert bytes to MB/GB etc in your head :)
 We have several internal scripts we use to parse the output to a more 
 readable output, and would be useful for it to be part of nodetool itself.



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


[jira] [Commented] (CASSANDRA-5078) save compaction merge counts in a system table

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784183#comment-13784183
 ] 

Jonathan Ellis commented on CASSANDRA-5078:
---

[~yanlantao], are you still working on this?

 save compaction merge counts in a system table
 --

 Key: CASSANDRA-5078
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5078
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matthew F. Dennis
Assignee: lantao yan
Priority: Minor
  Labels: lhf
 Attachments: 5078-v3.txt, 5078-v4.txt, patch1.patch


 we should save the compaction merge stats from CASSANDRA-4894 in the system 
 table and probably expose them via JMX (and nodetool)



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


[jira] [Commented] (CASSANDRA-5946) Commit Logs referencing deleted CFIDs not handled properly

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784185#comment-13784185
 ] 

Jonathan Ellis commented on CASSANDRA-5946:
---

Still working on this [~rbranson]?

 Commit Logs referencing deleted CFIDs not handled properly
 --

 Key: CASSANDRA-5946
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5946
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson
Assignee: Rick Branson
 Fix For: 1.2.11


 ERROR 19:44:38,377 Exception in thread Thread[COMMIT-LOG-WRITER,5,main]
 java.lang.NullPointerException
 at 
 org.apache.cassandra.db.commitlog.CommitLogAllocator.flushOldestTables(CommitLogAllocator.java:299)
 at 
 org.apache.cassandra.db.commitlog.CommitLogAllocator.fetchSegment(CommitLogAllocator.java:135)
 at 
 org.apache.cassandra.db.commitlog.CommitLog.activateNextSegment(CommitLog.java:333)
 at 
 org.apache.cassandra.db.commitlog.CommitLog.access$100(CommitLog.java:44)
 at 
 org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:377)
 at 
 org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:46)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.lang.Thread.run(Thread.java:679)
 Working up a test case / patch for this.



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


[jira] [Assigned] (CASSANDRA-5571) Reject bootstrapping endpoints that are already in the ring with different gossip data

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis reassigned CASSANDRA-5571:
-

Assignee: Tyler Hobbs  (was: Rick Branson)

 Reject bootstrapping endpoints that are already in the ring with different 
 gossip data
 --

 Key: CASSANDRA-5571
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5571
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Rick Branson
Assignee: Tyler Hobbs

 The ring can be silently broken by improperly bootstrapping an endpoint that 
 has an existing entry in the gossip table. In the case where a node attempts 
 to bootstrap with the same IP address as an existing ring member, the old 
 token metadata is dropped without warning, resulting in range shifts for the 
 cluster.
 This isn't so bad for non-vnode cases where, in general, tokens are 
 explicitly assigned, and a bootstrap on the same token would result in no 
 range shifts. For vnode cases, the convention is to just let nodes come up by 
 selecting their own tokens, and a bootstrap will override the existing tokens 
 for that endpoint.
 While there are some other issues open for adding an explicit rebootstrap 
 feature for vnode cases, given the changes in operator habits for vnode 
 rings, it seems a bit too easy to make this happen. Even more undesirable is 
 the fact that it's basically silent.
 This is a proposal for checking for this exact case: bootstraps on endpoints 
 with existing ring entries that have different hostIDs and/or tokens should 
 be rejected with an error message describing what happened and how to 
 override the safety check. It looks like the override can be supported using 
 the existing nodetool removenode -force.
 I can work up a patch for this.



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


[jira] [Commented] (CASSANDRA-5084) Cassandra should expose connected client state via JMX

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784190#comment-13784190
 ] 

Jonathan Ellis commented on CASSANDRA-5084:
---

Are you still working on this [~sureshsajja]?

 Cassandra should expose connected client state via JMX
 --

 Key: CASSANDRA-5084
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5084
 Project: Cassandra
  Issue Type: Improvement
Reporter: Robert Coli
Assignee: Suresh
Priority: Minor
  Labels: lhf
 Fix For: 1.2.11

 Attachments: 5084-v1.txt, cassandra-1.2-5084-native.patch, 
 trunk-5084-native.patch, trunk-5084.patch, trunk-5084-sept4.patch


 There is currently no good way to determine or estimate how many clients are 
 connected to a cassandra node without using netstat or (if using sync thrift 
 server) counting threads. There is also no way to understand what state any 
 given connection is in. People regularly come into #cassandra/cassandra-user@ 
 and ask how to get the equivalent of a MySQL SHOW FULL PROCESSLIST.
 While I understand that feature parity with SHOW FULL 
 PROCESSLIST/information_schema.processlist is unlikely, even a few basic 
 metrics like number of connected clients or number of active clients 
 would greatly help with this operational information need.



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


[2/3] git commit: change progressindicator members to CHM patch by jbellis for CASSANDRA-6129

2013-10-02 Thread jbellis
change progressindicator members to CHM
patch by jbellis for CASSANDRA-6129


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/70239e17
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/70239e17
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/70239e17

Branch: refs/heads/trunk
Commit: 70239e17b55b9e13b7f356683d78887f0c9a83c4
Parents: 546cbf5
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 12:42:06 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 12:42:10 2013 -0500

--
 src/java/org/apache/cassandra/tools/BulkLoader.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/70239e17/src/java/org/apache/cassandra/tools/BulkLoader.java
--
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java 
b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 8c732bc..cd3c7e1 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.cli.*;
@@ -84,8 +85,8 @@ public class BulkLoader
 // Return true when everything is at 100%
 static class ProgressIndicator implements StreamEventHandler
 {
-private final MapInetAddress, SessionInfo sessionsByHost = new 
HashMap();
-private final MapInetAddress, SetProgressInfo progressByHost = new 
HashMap();
+private final MapInetAddress, SessionInfo sessionsByHost = new 
ConcurrentHashMap();
+private final MapInetAddress, SetProgressInfo progressByHost = new 
ConcurrentHashMap();
 
 private long start;
 private long lastProgress;



[1/3] git commit: change progressindicator members to CHM patch by jbellis for CASSANDRA-6129

2013-10-02 Thread jbellis
Updated Branches:
  refs/heads/cassandra-2.0 546cbf5ff - 70239e17b
  refs/heads/trunk 7dd9ea2e8 - dca300d6c


change progressindicator members to CHM
patch by jbellis for CASSANDRA-6129


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/70239e17
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/70239e17
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/70239e17

Branch: refs/heads/cassandra-2.0
Commit: 70239e17b55b9e13b7f356683d78887f0c9a83c4
Parents: 546cbf5
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 12:42:06 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 12:42:10 2013 -0500

--
 src/java/org/apache/cassandra/tools/BulkLoader.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/70239e17/src/java/org/apache/cassandra/tools/BulkLoader.java
--
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java 
b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 8c732bc..cd3c7e1 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.cli.*;
@@ -84,8 +85,8 @@ public class BulkLoader
 // Return true when everything is at 100%
 static class ProgressIndicator implements StreamEventHandler
 {
-private final MapInetAddress, SessionInfo sessionsByHost = new 
HashMap();
-private final MapInetAddress, SetProgressInfo progressByHost = new 
HashMap();
+private final MapInetAddress, SessionInfo sessionsByHost = new 
ConcurrentHashMap();
+private final MapInetAddress, SetProgressInfo progressByHost = new 
ConcurrentHashMap();
 
 private long start;
 private long lastProgress;



[3/3] git commit: Merge branch 'cassandra-2.0' into trunk

2013-10-02 Thread jbellis
Merge branch 'cassandra-2.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/dca300d6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/dca300d6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/dca300d6

Branch: refs/heads/trunk
Commit: dca300d6c546f99c52778f0db597721365233d42
Parents: 7dd9ea2 70239e1
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 12:42:38 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 12:42:38 2013 -0500

--
 src/java/org/apache/cassandra/tools/BulkLoader.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--




[jira] [Comment Edited] (CASSANDRA-5078) save compaction merge counts in a system table

2013-10-02 Thread lantao yan (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784212#comment-13784212
 ] 

lantao yan edited comment on CASSANDRA-5078 at 10/2/13 6:02 PM:


last time i checked the trunk, my code is already there. I am wondering what 
has happened. i will double check that.


was (Author: yanlantao):
last time i checked the trunk, my code is already there. I am wondering what 
has happened.

 save compaction merge counts in a system table
 --

 Key: CASSANDRA-5078
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5078
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matthew F. Dennis
Assignee: lantao yan
Priority: Minor
  Labels: lhf
 Attachments: 5078-v3.txt, 5078-v4.txt, patch1.patch


 we should save the compaction merge stats from CASSANDRA-4894 in the system 
 table and probably expose them via JMX (and nodetool)



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


[jira] [Commented] (CASSANDRA-5078) save compaction merge counts in a system table

2013-10-02 Thread lantao yan (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784239#comment-13784239
 ] 

lantao yan commented on CASSANDRA-5078:
---

sorry, i think i was wrong. I will read the code and consider Yuki's 
suggestion. it is still a little bit weird to me if two threads can work on the 
same column family.

 save compaction merge counts in a system table
 --

 Key: CASSANDRA-5078
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5078
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matthew F. Dennis
Assignee: lantao yan
Priority: Minor
  Labels: lhf
 Attachments: 5078-v3.txt, 5078-v4.txt, patch1.patch


 we should save the compaction merge stats from CASSANDRA-4894 in the system 
 table and probably expose them via JMX (and nodetool)



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


[jira] [Commented] (CASSANDRA-5078) save compaction merge counts in a system table

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784233#comment-13784233
 ] 

Jonathan Ellis commented on CASSANDRA-5078:
---

Hmm, if so this ticket did not get mentioned in a commit message or in CHANGES.

 save compaction merge counts in a system table
 --

 Key: CASSANDRA-5078
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5078
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matthew F. Dennis
Assignee: lantao yan
Priority: Minor
  Labels: lhf
 Attachments: 5078-v3.txt, 5078-v4.txt, patch1.patch


 we should save the compaction merge stats from CASSANDRA-4894 in the system 
 table and probably expose them via JMX (and nodetool)



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


[jira] [Commented] (CASSANDRA-6109) Consider coldness in STCS compaction

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784245#comment-13784245
 ] 

Jonathan Ellis commented on CASSANDRA-6109:
---

I guess whether hotness or overlap is a more important criterion depends on 
your goal:
# prioritizing by hotness helps speed reads up more, especially when you have a 
lot of cold data sitting around
# prioritizing by overlap ratio reduces disk space and helps throw away 
obsolete cells faster

I was hoping to tackle #1 here, but maybe that needs a separate strategy a la 
CASSANDRA-5560.

For #2, CASSANDRA-5906 adds a HyperLogLog component that does a fantastic job 
of letting us estimate overlap ratios.



 Consider coldness in STCS compaction
 

 Key: CASSANDRA-6109
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
 Fix For: 2.0.2


 I see two options:
 # Don't compact cold sstables at all
 # Compact cold sstables only if there is nothing more important to compact
 The latter is better if you have cold data that may become hot again...  but 
 it's confusing if you have a workload such that you can't keep up with *all* 
 compaction, but you can keep up with hot sstable.  (Compaction backlog stat 
 becomes useless since we fall increasingly behind.)



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


[jira] [Updated] (CASSANDRA-1337) parallelize fetching rows for low-cardinality indexes

2013-10-02 Thread Tyler Hobbs (JIRA)

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

Tyler Hobbs updated CASSANDRA-1337:
---

Attachment: 0001-Concurrent-range-and-2ary-index-subqueries.patch

Patch 0001 is a somewhat re-worked version of David's patch against trunk.

Breaking down estimates by the filter type was neglecting all non-compact cql3 
queries (and some compact ones).  A simpler way to estimate the number of 
results is to break down the queries into four groups:
* Secondary index query: use the mean columns from the index CF (one column per 
data row or cql3 data row)
* Non-cql3 range query: use the estimated keys
* cql3 range query, compact (compact storage or single-component primary key): 
use the estimated keys
* cql3 range query, non-compact: use (estimated_keys * mean_columns) / 
(number_of_defined_columns)

The last case is the least accurate.  When collections are involved, it will 
overestimate the number of cql3 rows that will be returned, meaning additional 
ranges may need to be queried, but I think this is an acceptable optimization 
degradation.

Another change from David's patch is that if an insufficient number of results 
are fetched by the first round, the concurrency factor will be recalculated 
based on the results we've seen so far instead of simply being set to 1.

I wasn't sure where to add tests for this; would dtests be the best place?

 parallelize fetching rows for low-cardinality indexes
 -

 Key: CASSANDRA-1337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1337
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
Priority: Minor
 Fix For: 2.1

 Attachments: 0001-Concurrent-range-and-2ary-index-subqueries.patch, 
 1137-bugfix.patch, 1337.patch, 1337-v4.patch, 
 ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1337-scan-concurrently-depending-on-num-rows.txt,
  CASSANDRA-1337.patch

   Original Estimate: 8h
  Remaining Estimate: 8h

 currently, we read the indexed rows from the first node (in partitioner 
 order); if that does not have enough matching rows, we read the rows from the 
 next, and so forth.
 we should use the statistics fom CASSANDRA-1155 to query multiple nodes in 
 parallel, such that we have a high chance of getting enough rows w/o having 
 to do another round of queries (but, if our estimate is incorrect, we do need 
 to loop and do more rounds until we have enough data or we have fetched from 
 each node).



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


[jira] [Comment Edited] (CASSANDRA-6109) Consider coldness in STCS compaction

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784245#comment-13784245
 ] 

Jonathan Ellis edited comment on CASSANDRA-6109 at 10/2/13 6:25 PM:


I guess whether hotness or overlap is a more important criterion depends on 
your goal:
# prioritizing by hotness helps speed reads up more, especially when you have a 
lot of cold data sitting around
# prioritizing by overlap ratio reduces disk space and helps throw away 
obsolete cells faster

I was hoping to tackle #1 here, but maybe that needs a separate strategy a la 
CASSANDRA-5561.

For #2, CASSANDRA-5906 adds a HyperLogLog component that does a fantastic job 
of letting us estimate overlap ratios.




was (Author: jbellis):
I guess whether hotness or overlap is a more important criterion depends on 
your goal:
# prioritizing by hotness helps speed reads up more, especially when you have a 
lot of cold data sitting around
# prioritizing by overlap ratio reduces disk space and helps throw away 
obsolete cells faster

I was hoping to tackle #1 here, but maybe that needs a separate strategy a la 
CASSANDRA-5560.

For #2, CASSANDRA-5906 adds a HyperLogLog component that does a fantastic job 
of letting us estimate overlap ratios.



 Consider coldness in STCS compaction
 

 Key: CASSANDRA-6109
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
 Fix For: 2.0.2


 I see two options:
 # Don't compact cold sstables at all
 # Compact cold sstables only if there is nothing more important to compact
 The latter is better if you have cold data that may become hot again...  but 
 it's confusing if you have a workload such that you can't keep up with *all* 
 compaction, but you can keep up with hot sstable.  (Compaction backlog stat 
 becomes useless since we fall increasingly behind.)



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


[jira] [Updated] (CASSANDRA-1337) parallelize fetching rows for low-cardinality indexes

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-1337:
--

Reviewer: Jonathan Ellis  (was: Sylvain Lebresne)

 parallelize fetching rows for low-cardinality indexes
 -

 Key: CASSANDRA-1337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1337
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
Priority: Minor
 Fix For: 2.1

 Attachments: 0001-Concurrent-range-and-2ary-index-subqueries.patch, 
 1137-bugfix.patch, 1337.patch, 1337-v4.patch, 
 ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1337-scan-concurrently-depending-on-num-rows.txt,
  CASSANDRA-1337.patch

   Original Estimate: 8h
  Remaining Estimate: 8h

 currently, we read the indexed rows from the first node (in partitioner 
 order); if that does not have enough matching rows, we read the rows from the 
 next, and so forth.
 we should use the statistics fom CASSANDRA-1155 to query multiple nodes in 
 parallel, such that we have a high chance of getting enough rows w/o having 
 to do another round of queries (but, if our estimate is incorrect, we do need 
 to loop and do more rounds until we have enough data or we have fetched from 
 each node).



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


[jira] [Commented] (CASSANDRA-6132) CL.ANY writes can still time out

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784279#comment-13784279
 ] 

Aleksey Yeschenko commented on CASSANDRA-6132:
--

Huh, are you sure? It doesn't build for me (WriteCallbackInfo?).



 CL.ANY writes can still time out
 

 Key: CASSANDRA-6132
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6132
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 1.2.11, 2.0.2

 Attachments: 6132.txt


 If we know that all replicas are down at the beginning of a mutation, we will 
 write a hint and return success.
 But if we do not, we will attemp to write to replicas, time out, return 
 failure, and then write a hint, violating our contract that (unless the 
 coordinator goes down), writes at CL.ANY should always succeed.



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


[jira] [Updated] (CASSANDRA-2848) Make the Client API support passing down timeouts

2013-10-02 Thread sankalp kohli (JIRA)

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

sankalp kohli updated CASSANDRA-2848:
-

Component/s: Core

 Make the Client API support passing down timeouts
 -

 Key: CASSANDRA-2848
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2848
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Priority: Minor

 Having a max server RPC timeout is good for worst case, but many applications 
 that have middleware in front of Cassandra, might have higher timeout 
 requirements. In a fail fast environment, if my application starting at say 
 the front-end, only has 20ms to process a request, and it must connect to X 
 services down the stack, by the time it hits Cassandra, we might only have 
 10ms. I propose we provide the ability to specify the timeout on each call we 
 do optionally.



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


[jira] [Commented] (CASSANDRA-6121) CASS 2.0, possibly 1.2.8 as well: Secondary Indexes not working

2013-10-02 Thread Constance Eustace (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784280#comment-13784280
 ] 

Constance Eustace commented on CASSANDRA-6121:
--

We also have encountered queries against indexex/indices that seem to return 
data from other keyspaces(!)...

We are exploring disabling of rowcaching, I think that has fixed the problem, 
we will be attempting to evaluate the read performance impact...

 CASS 2.0, possibly 1.2.8 as well: Secondary Indexes not working
 ---

 Key: CASSANDRA-6121
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6121
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: AWS ebs striped quad-volume for data directories, ubuntu 
 12.10. Currently single-node, but also possibly seen in two-node 
 configuration.
Reporter: Constance Eustace
 Fix For: 1.2.8, 2.0.1


 I will attach the schema we are using. We are using CQL3, fed via the 
 cass-jdbc driver project. 
 We are using off-heap JNA-enabled key and row caching.
 We implement an entity model using cassandra wide/sparse row. So an 
 entityid is the rowkey, and various properties are stored with property 
 names as the column key and various value information (type, value(s)), etc. 
 Some of these sparse columns are indexes so there can be searches on the 
 values. 
 We have a fairly large number of indexes. 
 Data is populated using heavy batch intakes (1.2 million row keys done in 
 about 16 minutes).
 We will attempt to reproduce reliably, get stats, logs, traces. Filing the 
 bug for now as a placeholder.
 These 1.2 million rowkey updates are split into individual batches of about 
 200 statements, with a commonly shared timestamp specified for the batch so 
 that update contention can be dealt with. 
 I have seen the previous filed bugs on compaction on TTL columns (not used by 
 us) and rowkey caching having impacts on the indexes. We may attempt 
 experiments where we do not use rowkey caching, toggling JNA/offheap, etc. 
 Any advice would be appreciated for detecting index failure...
 Our schema: (we have another 8-10 near copies of this keyspace that split the 
 data for vendors/storefronts/etc)
 CREATE KEYSPACE internal_submission WITH REPLICATION= { 
 'class':'SimpleStrategy', 
 'replication_factor':%=node.ingest.db.replication_factor% };
 CREATE TABLE internal_submission.Relation (ParentID text,ChildID 
 text,GraphType text,Info maptext,text,PRIMARY KEY (ParentID,ChildID)) with 
 caching = 'all';
 CREATE TABLE internal_submission.RelationBACKREF (ChildID text,ParentID 
 text,PRIMARY KEY (ChildID,ParentID)) with caching = 'all';
 CREATE TABLE internal_submission.Blob (BlobID text,Type text,SubType 
 text,Encoding maptext,text,BlobData blob,PRIMARY KEY (BlobID)) with caching 
 = 'keys_only';
 CREATE TABLE internal_submission.Entity_Job (e_EntID text,e_EntName 
 text,e_EntType text,e_EntLinks text,p_Prop text,p_Storage text,p_PropID 
 text,p_Flags text,p_Val text,p_ValType text,p_ValUnit text,p_ValLang 
 text,p_ValLinks text,p_Vars text,p_PropLinks text,p_SubEnts text,PartnerID 
 text,UserID text,SubmitDate bigint,SourceIP text,SubmitEvent text,Size 
 int,Describes text,Version text,IngestStatus text,IngestStatusDetail 
 text,ReferenceID text,DNDCondition text,PRIMARY KEY (e_EntID,p_Prop)) with 
 caching = 'all';
 CREATE TABLE internal_submission.Processing (EntityID text,InProcess 
 counter,Complete counter,Success counter,Fail counter,Warn counter,Redo 
 counter,Hold counter,PRIMARY KEY (EntityID)) with caching = 'all';
 CREATE TABLE internal_submission.Entity_Asset (e_EntID text,e_EntName 
 text,e_EntType text,e_EntLinks text,p_Prop text,p_Storage text,p_PropID 
 text,p_Flags text,p_Val text,p_ValType text,p_ValUnit text,p_ValLang 
 text,p_ValLinks text,p_Vars text,p_PropLinks text,p_SubEnts text,IngestStatus 
 text,IngestStatusDetail text,PRIMARY KEY (e_EntID,p_Prop)) with caching = 
 'all';
 CREATE TABLE internal_submission.Entity_MetaDataDef (e_EntID text,e_EntName 
 text,e_EntType text,e_EntLinks text,p_Prop text,p_Storage text,p_PropID 
 text,p_Flags text,p_Val text,p_ValType text,p_ValUnit text,p_ValLang 
 text,p_ValLinks text,p_Vars text,p_PropLinks text,p_SubEnts text,PRIMARY KEY 
 (e_EntID,p_Prop)) with caching = 'all';
 CREATE TABLE internal_submission.Entity_HierarchyDef (e_EntID text,e_EntName 
 text,e_EntType text,e_EntLinks text,p_Prop text,p_Storage text,p_PropID 
 text,p_Flags text,p_Val text,p_ValType text,p_ValUnit text,p_ValLang 
 text,p_ValLinks text,p_Vars text,p_PropLinks text,p_SubEnts text,Describes 
 text,Version text,PRIMARY KEY (e_EntID,p_Prop)) with caching = 'all';
 CREATE TABLE internal_submission.Entity_CategoryDef (e_EntID text,e_EntName 
 text,e_EntType text,e_EntLinks text,p_Prop 

[jira] [Commented] (CASSANDRA-2848) Make the Client API support passing down timeouts

2013-10-02 Thread sankalp kohli (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784287#comment-13784287
 ] 

sankalp kohli commented on CASSANDRA-2848:
--

This will be very useful for read queries with different time outs. 

 Make the Client API support passing down timeouts
 -

 Key: CASSANDRA-2848
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2848
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Priority: Minor

 Having a max server RPC timeout is good for worst case, but many applications 
 that have middleware in front of Cassandra, might have higher timeout 
 requirements. In a fail fast environment, if my application starting at say 
 the front-end, only has 20ms to process a request, and it must connect to X 
 services down the stack, by the time it hits Cassandra, we might only have 
 10ms. I propose we provide the ability to specify the timeout on each call we 
 do optionally.



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


[jira] [Created] (CASSANDRA-6135) Add beforeChange Notification to Gossiper State.

2013-10-02 Thread Benjamin Coverston (JIRA)
Benjamin Coverston created CASSANDRA-6135:
-

 Summary: Add beforeChange Notification to Gossiper State.
 Key: CASSANDRA-6135
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6135
 Project: Cassandra
  Issue Type: New Feature
Reporter: Benjamin Coverston
 Attachments: 
0001-New-Gossiper-notification-to-IEndpointStateChangeSub.patch

We would like an internal notification to be fired before state changes happen 
so we can intercept them, and in some cases defer them.



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


[jira] [Updated] (CASSANDRA-6135) Add beforeChange Notification to Gossiper State.

2013-10-02 Thread Benjamin Coverston (JIRA)

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

Benjamin Coverston updated CASSANDRA-6135:
--

Attachment: 0001-New-Gossiper-notification-to-IEndpointStateChangeSub.patch

Attached is Sergio's patch for 1.2

 Add beforeChange Notification to Gossiper State.
 

 Key: CASSANDRA-6135
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6135
 Project: Cassandra
  Issue Type: New Feature
Reporter: Benjamin Coverston
 Attachments: 
 0001-New-Gossiper-notification-to-IEndpointStateChangeSub.patch


 We would like an internal notification to be fired before state changes 
 happen so we can intercept them, and in some cases defer them.



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


[jira] [Commented] (CASSANDRA-1337) parallelize fetching rows for low-cardinality indexes

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784290#comment-13784290
 ] 

Jonathan Ellis commented on CASSANDRA-1337:
---

Pushed some super minor cleanup to 
https://github.com/jbellis/cassandra/commits/1337.

Question left in my mind is, do we want to shoot for exactly enough concurrent 
requests, on average?  Would imply that half the time we need to do an extra 
round.  ISTM we probably want to give ourselves a margin of error.

 parallelize fetching rows for low-cardinality indexes
 -

 Key: CASSANDRA-1337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1337
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
Priority: Minor
 Fix For: 2.1

 Attachments: 0001-Concurrent-range-and-2ary-index-subqueries.patch, 
 1137-bugfix.patch, 1337.patch, 1337-v4.patch, 
 ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1337-scan-concurrently-depending-on-num-rows.txt,
  CASSANDRA-1337.patch

   Original Estimate: 8h
  Remaining Estimate: 8h

 currently, we read the indexed rows from the first node (in partitioner 
 order); if that does not have enough matching rows, we read the rows from the 
 next, and so forth.
 we should use the statistics fom CASSANDRA-1155 to query multiple nodes in 
 parallel, such that we have a high chance of getting enough rows w/o having 
 to do another round of queries (but, if our estimate is incorrect, we do need 
 to loop and do more rounds until we have enough data or we have fetched from 
 each node).



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


[jira] [Comment Edited] (CASSANDRA-2848) Make the Client API support passing down timeouts

2013-10-02 Thread sankalp kohli (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784287#comment-13784287
 ] 

sankalp kohli edited comment on CASSANDRA-2848 at 10/2/13 7:00 PM:
---

This will be very useful for read queries with different time outs. This will 
also be useful when changing the timeout in the client. Then we don't need to 
change it in Cassandra. 


was (Author: kohlisankalp):
This will be very useful for read queries with different time outs. 

 Make the Client API support passing down timeouts
 -

 Key: CASSANDRA-2848
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2848
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Priority: Minor

 Having a max server RPC timeout is good for worst case, but many applications 
 that have middleware in front of Cassandra, might have higher timeout 
 requirements. In a fail fast environment, if my application starting at say 
 the front-end, only has 20ms to process a request, and it must connect to X 
 services down the stack, by the time it hits Cassandra, we might only have 
 10ms. I propose we provide the ability to specify the timeout on each call we 
 do optionally.



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


[jira] [Commented] (CASSANDRA-1337) parallelize fetching rows for low-cardinality indexes

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784296#comment-13784296
 ] 

Jonathan Ellis commented on CASSANDRA-1337:
---

And yeah, dtests is probably the only sane place to test.

 parallelize fetching rows for low-cardinality indexes
 -

 Key: CASSANDRA-1337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1337
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
Priority: Minor
 Fix For: 2.1

 Attachments: 0001-Concurrent-range-and-2ary-index-subqueries.patch, 
 1137-bugfix.patch, 1337.patch, 1337-v4.patch, 
 ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1337-scan-concurrently-depending-on-num-rows.txt,
  CASSANDRA-1337.patch

   Original Estimate: 8h
  Remaining Estimate: 8h

 currently, we read the indexed rows from the first node (in partitioner 
 order); if that does not have enough matching rows, we read the rows from the 
 next, and so forth.
 we should use the statistics fom CASSANDRA-1155 to query multiple nodes in 
 parallel, such that we have a high chance of getting enough rows w/o having 
 to do another round of queries (but, if our estimate is incorrect, we do need 
 to loop and do more rounds until we have enough data or we have fetched from 
 each node).



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


[jira] [Commented] (CASSANDRA-6109) Consider coldness in STCS compaction

2013-10-02 Thread Tyler Hobbs (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784300#comment-13784300
 ] 

Tyler Hobbs commented on CASSANDRA-6109:


I think I have some clearer ideas about how to do this now.  We should be able 
to combine hotness and overlap concerns at the different levels.

At level (1), avoid compacting comparatively cold data by dropping sstables 
from buckets when their hotness is less than, say, 25% of the bucket average 
(this avoids the low-variance problem of using the stddev).  If the bucket 
falls below the min compaction threshold, ignore it (to make sure we're 
compacting enough sstables at once).

At level (2), submit the hottest bucket to the executor for compaction.

The average number of sstables hit per-read is actually a decent measure for 
prioritizing compactions at the executor level.  At level (3), we can combine 
that with the bucket hotness to get a rough idea of how many individual sstable 
reads per second we could save by compacting a given bucket (hotness * 
avg_sstables_per_read).  Prioritize compaction tasks in the queue based on this 
measure.

That should give us a nice balance of not compacting cold data and prioritizing 
compaction of the most read and most fragmented sstables.

 Consider coldness in STCS compaction
 

 Key: CASSANDRA-6109
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
 Fix For: 2.0.2


 I see two options:
 # Don't compact cold sstables at all
 # Compact cold sstables only if there is nothing more important to compact
 The latter is better if you have cold data that may become hot again...  but 
 it's confusing if you have a workload such that you can't keep up with *all* 
 compaction, but you can keep up with hot sstable.  (Compaction backlog stat 
 becomes useless since we fall increasingly behind.)



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


[jira] [Updated] (CASSANDRA-6132) CL.ANY writes can still time out

2013-10-02 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-6132:
--

Attachment: 6132-v2.txt

v2

 CL.ANY writes can still time out
 

 Key: CASSANDRA-6132
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6132
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 1.2.11, 2.0.2

 Attachments: 6132.txt, 6132-v2.txt


 If we know that all replicas are down at the beginning of a mutation, we will 
 write a hint and return success.
 But if we do not, we will attemp to write to replicas, time out, return 
 failure, and then write a hint, violating our contract that (unless the 
 coordinator goes down), writes at CL.ANY should always succeed.



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


[jira] [Commented] (CASSANDRA-1337) parallelize fetching rows for low-cardinality indexes

2013-10-02 Thread Tyler Hobbs (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784311#comment-13784311
 ] 

Tyler Hobbs commented on CASSANDRA-1337:


bq. Pushed some super minor cleanup to 
https://github.com/jbellis/cassandra/commits/1337.

In your OCD commit, replacing the haveSufficientRows/break behavior with a 
return means that we won't wait on the repair futures, which I believe is 
incorrect.

bq. Question left in my mind is, do we want to shoot for exactly enough 
concurrent requests, on average? Would imply that half the time we need to do 
an extra round. ISTM we probably want to give ourselves a margin of error.

True.  Perhaps we should decrease our estimate of rows per range by, say, 10%, 
and use Math.ceil() instead of Math.round() for the concurrency factor 
calculation.

 parallelize fetching rows for low-cardinality indexes
 -

 Key: CASSANDRA-1337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1337
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
Priority: Minor
 Fix For: 2.1

 Attachments: 0001-Concurrent-range-and-2ary-index-subqueries.patch, 
 1137-bugfix.patch, 1337.patch, 1337-v4.patch, 
 ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1337-scan-concurrently-depending-on-num-rows.txt,
  CASSANDRA-1337.patch

   Original Estimate: 8h
  Remaining Estimate: 8h

 currently, we read the indexed rows from the first node (in partitioner 
 order); if that does not have enough matching rows, we read the rows from the 
 next, and so forth.
 we should use the statistics fom CASSANDRA-1155 to query multiple nodes in 
 parallel, such that we have a high chance of getting enough rows w/o having 
 to do another round of queries (but, if our estimate is incorrect, we do need 
 to loop and do more rounds until we have enough data or we have fetched from 
 each node).



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


[jira] [Commented] (CASSANDRA-6109) Consider coldness in STCS compaction

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784323#comment-13784323
 ] 

Jonathan Ellis commented on CASSANDRA-6109:
---

SGTM.

 Consider coldness in STCS compaction
 

 Key: CASSANDRA-6109
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Tyler Hobbs
 Fix For: 2.0.2


 I see two options:
 # Don't compact cold sstables at all
 # Compact cold sstables only if there is nothing more important to compact
 The latter is better if you have cold data that may become hot again...  but 
 it's confusing if you have a workload such that you can't keep up with *all* 
 compaction, but you can keep up with hot sstable.  (Compaction backlog stat 
 becomes useless since we fall increasingly behind.)



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


[jira] [Commented] (CASSANDRA-6116) /etc/init.d/cassandra stop and service don't work

2013-10-02 Thread Sven Delmas (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784358#comment-13784358
 ] 

Sven Delmas commented on CASSANDRA-6116:


The issue seems to be that /var/run/cassandra is not being created during 
installation. So no pid file to use for stop. Interestingly enough the status 
call still fails to match the regular expression being used. I suspect that 
cmdline has changed compared to when jsvc was still used.

 /etc/init.d/cassandra stop and service don't work
 -

 Key: CASSANDRA-6116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6116
 Project: Cassandra
  Issue Type: Bug
  Components: Packaging
Reporter: Cathy Daw
Assignee: Eric Evans
Priority: Minor

 These use to work in 2.0.0 and appears to be introduced in 2.0.1
 Test Scenario
 {noformat}
 # Start Server
 automaton@ip-10-171-39-230:~$ sudo service cassandra start
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
 # Check Status
 automaton@ip-10-171-39-230:~$ nodetool status
 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  AddressLoad   Tokens  Owns   Host ID  
  Rack
 UN  127.0.0.1  81.72 KB   256 100.0%  
 e40ef77c-9cf7-4e27-b651-ede3b7269019  rack1
 # Check Status of service
 automaton@ip-10-171-39-230:~$ sudo service cassandra status
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
  * Cassandra is not running
 # Stop Server
 automaton@ip-10-171-39-230:~$ sudo service cassandra stop
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
 # Verify Server is no longer up
 automaton@ip-10-171-39-230:~$ nodetool status
 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  AddressLoad   Tokens  Owns   Host ID  
  Rack
 UN  127.0.0.1  81.72 KB   256 100.0%  
 e40ef77c-9cf7-4e27-b651-ede3b7269019  rack1
 {noformat}
 Installation Instructions
 {noformat}
 wget http://people.apache.org/~slebresne/cassandra_2.0.1_all.deb
 sudo dpkg -i cassandra_2.0.1_all.deb # Error about dependencies
 sudo apt-get -f install
 sudo dpkg -i cassandra_2.0.1_all.deb
 {noformat}



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


[jira] [Created] (CASSANDRA-6136) CQL should not allow an empty string as column identifier

2013-10-02 Thread JIRA
Michaël Figuière created CASSANDRA-6136:
---

 Summary: CQL should not allow an empty string as column identifier
 Key: CASSANDRA-6136
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6136
 Project: Cassandra
  Issue Type: Bug
Reporter: Michaël Figuière
Priority: Minor


CQL currently allows users to create a table with an empty string as column 
identifier:

{code}
CREATE TABLE t (k int primary key,  int);
{code}

Which results in the following table:

{code}
CREATE TABLE t (
  k int,
   int,
  PRIMARY KEY (k)
) WITH
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=864000 AND
  index_interval=128 AND
  read_repair_chance=0.10 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'SnappyCompressor'};
{code}

Empty strings are not allowed for keyspace and table identifiers though.

I guess it's just a case that we haven't covered. Of course making it illegal 
in a future version would be a breaking change, but nobody serious would 
manually have chosen such an identifier...



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


[jira] [Commented] (CASSANDRA-5683) Write timeout in multi-dc environment

2013-10-02 Thread sankalp kohli (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784398#comment-13784398
 ] 

sankalp kohli commented on CASSANDRA-5683:
--

Timeouts will differ based on a combination of many things like operation 
time(read, write), consistency level(multi-DC or not) and whether it is a batch 
or single operation. 
Also you will want different timeouts for  multi-DC hops vs within the same DC. 
Instead of having so many properties for each combination, we should let client 
decide the timeout and should propagate that timeout. This is because client 
knows which combination it is using. We might also allow passing separate 
timeouts for within DC and multi-DC reads or writes. 
This will be very useful for reads as there is no point working on timed out 
queries. 

 Write timeout in multi-dc environment 
 --

 Key: CASSANDRA-5683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5683
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.1.9
 Environment: apache cassandra 1.1.9
Reporter: Boole Guo
  Labels: write

 When writing in mutil-dc environment, there are many timeout exception. As I 
 know, this version have do good to reduce network bandwidth. Can we assign 
 rpc timeout when writing like consistenylevel?



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


[jira] [Comment Edited] (CASSANDRA-6136) CQL should not allow an empty string as column identifier

2013-10-02 Thread JIRA

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6136?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784400#comment-13784400
 ] 

Michaël Figuière edited comment on CASSANDRA-6136 at 10/2/13 8:37 PM:
--

Looks like it's used in {{system.IndexInfo}} actually:

{code}
cqlsh DESC TABLE system.IndexInfo

CREATE TABLE IndexInfo (
  table_name text,
  index_name text,
   blob,
  PRIMARY KEY (table_name, index_name)
) WITH COMPACT STORAGE AND
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='indexes that have been completed' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=0 AND
  index_interval=128 AND
  read_repair_chance=0.00 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};
{code}

Would it be the reason for that to be allowed?


was (Author: mfiguiere):
Looks like its used in {{system.IndexInfo}} actually:

{code}
cqlsh DESC TABLE system.IndexInfo

CREATE TABLE IndexInfo (
  table_name text,
  index_name text,
   blob,
  PRIMARY KEY (table_name, index_name)
) WITH COMPACT STORAGE AND
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='indexes that have been completed' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=0 AND
  index_interval=128 AND
  read_repair_chance=0.00 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};
{code}

Would it be the reason for that to be allowed?

 CQL should not allow an empty string as column identifier
 -

 Key: CASSANDRA-6136
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6136
 Project: Cassandra
  Issue Type: Bug
Reporter: Michaël Figuière
Priority: Minor

 CQL currently allows users to create a table with an empty string as column 
 identifier:
 {code}
 CREATE TABLE t (k int primary key,  int);
 {code}
 Which results in the following table:
 {code}
 CREATE TABLE t (
   k int,
int,
   PRIMARY KEY (k)
 ) WITH
   bloom_filter_fp_chance=0.01 AND
   caching='KEYS_ONLY' AND
   comment='' AND
   dclocal_read_repair_chance=0.00 AND
   gc_grace_seconds=864000 AND
   index_interval=128 AND
   read_repair_chance=0.10 AND
   replicate_on_write='true' AND
   populate_io_cache_on_flush='false' AND
   default_time_to_live=0 AND
   speculative_retry='NONE' AND
   memtable_flush_period_in_ms=0 AND
   compaction={'class': 'SizeTieredCompactionStrategy'} AND
   compression={'sstable_compression': 'SnappyCompressor'};
 {code}
 Empty strings are not allowed for keyspace and table identifiers though.
 I guess it's just a case that we haven't covered. Of course making it illegal 
 in a future version would be a breaking change, but nobody serious would 
 manually have chosen such an identifier...



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


[jira] [Commented] (CASSANDRA-6136) CQL should not allow an empty string as column identifier

2013-10-02 Thread JIRA

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6136?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784400#comment-13784400
 ] 

Michaël Figuière commented on CASSANDRA-6136:
-

Looks like its used in {{system.IndexInfo}} actually:

{code}
cqlsh DESC TABLE system.IndexInfo

CREATE TABLE IndexInfo (
  table_name text,
  index_name text,
   blob,
  PRIMARY KEY (table_name, index_name)
) WITH COMPACT STORAGE AND
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='indexes that have been completed' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=0 AND
  index_interval=128 AND
  read_repair_chance=0.00 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};
{code}

Would it be the reason for that to be allowed?

 CQL should not allow an empty string as column identifier
 -

 Key: CASSANDRA-6136
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6136
 Project: Cassandra
  Issue Type: Bug
Reporter: Michaël Figuière
Priority: Minor

 CQL currently allows users to create a table with an empty string as column 
 identifier:
 {code}
 CREATE TABLE t (k int primary key,  int);
 {code}
 Which results in the following table:
 {code}
 CREATE TABLE t (
   k int,
int,
   PRIMARY KEY (k)
 ) WITH
   bloom_filter_fp_chance=0.01 AND
   caching='KEYS_ONLY' AND
   comment='' AND
   dclocal_read_repair_chance=0.00 AND
   gc_grace_seconds=864000 AND
   index_interval=128 AND
   read_repair_chance=0.10 AND
   replicate_on_write='true' AND
   populate_io_cache_on_flush='false' AND
   default_time_to_live=0 AND
   speculative_retry='NONE' AND
   memtable_flush_period_in_ms=0 AND
   compaction={'class': 'SizeTieredCompactionStrategy'} AND
   compression={'sstable_compression': 'SnappyCompressor'};
 {code}
 Empty strings are not allowed for keyspace and table identifiers though.
 I guess it's just a case that we haven't covered. Of course making it illegal 
 in a future version would be a breaking change, but nobody serious would 
 manually have chosen such an identifier...



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


[jira] [Comment Edited] (CASSANDRA-6116) /etc/init.d/cassandra stop and service don't work

2013-10-02 Thread Sven Delmas (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784358#comment-13784358
 ] 

Sven Delmas edited comment on CASSANDRA-6116 at 10/2/13 8:46 PM:
-

The issue seems to be that /var/run/cassandra is not being created during 
installation. So no pid file to use for stop. Interestingly enough the status 
call still fails to match the regular expression being used. I suspect that 
cmdline has changed compared to when jsvc was still used.

The output from /proc/$pid/cmdline looks like this:

java-ea-javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar-XX:+UseThreadPriorities-XX:ThreadPriorityPolicy=42-Xms998M-Xmx998M-Xmn100M-XX:+HeapDumpOnOutOfMemoryError-Xss256k-XX:+UseParNewGC-XX:+UseConcMarkSweepGC-XX:+CMSParallelRemarkEnabled-XX:SurvivorRatio=8-XX:MaxTenuringThreshold=1-XX:CMSInitiatingOccupancyFraction=75-XX:+UseCMSInitiatingOccupancyOnly-XX:+UseTLAB-XX:+UseCondCardMark-Djava.net.preferIPv4Stack=true-Dcom.sun.management.jmxremote.port=7199-Dcom.sun.management.jmxremote.ssl=false-Dcom.sun.management.jmxremote.authenticate=false-Dlog4j.configuration=log4j-server.properties-Dlog4j.defaultInitOverride=true-Dcassandra-pidfile=/var/run/cassandra/cassandra.pid-cp/etc/cassandra:/usr/share/cassandra/lib/antlr-3.2.jar:/usr/share/cassandra/lib/commons-cli-1.1.jar:/usr/share/cassandra/lib/commons-codec-1.2.jar:/usr/share/cassandra/lib/commons-lang3-3.1.jar:/usr/share/cassandra/lib/compress-lzf-0.8.4.jar:/usr/share/cassandra/lib/concurrentlinkedhashmap-lru-1.3.jar:/usr/share/cassandra/lib/disruptor-3.0.1.jar:/usr/share/cassandra/lib/guava-15.0.jar:/usr/share/cassandra/lib/high-scale-lib-1.1.2.jar:/usr/share/cassandra/lib/jackson-core-asl-1.9.2.jar:/usr/share/cassandra/lib/jackson-mapper-asl-1.9.2.jar:/usr/share/cassandra/lib/jamm-0.2.5.jar:/usr/share/cassandra/lib/jbcrypt-0.3m.jar:/usr/share/cassandra/lib/jline-1.0.jar:/usr/share/cassandra/lib/json-simple-1.1.jar:/usr/share/cassandra/lib/libthrift-0.9.1.jar:/usr/share/cassandra/lib/log4j-1.2.16.jar:/usr/share/cassandra/lib/lz4-1.1.0.jar:/usr/share/cassandra/lib/metrics-core-2.2.0.jar:/usr/share/cassandra/lib/netty-3.6.6.Final.jar:/usr/share/cassandra/lib/servlet-api-2.5-20081211.jar:/usr/share/cassandra/lib/slf4j-api-1.7.2.jar:/usr/share/cassandra/lib/slf4j-log4j12-1.7.2.jar:/usr/share/cassandra/lib/snakeyaml-1.11.jar:/usr/share/cassandra/lib/snappy-java-1.0.5.jar:/usr/share/cassandra/lib/snaptree-0.1.jar:/usr/share/cassandra/lib/thrift-server-0.3.2.jar:/usr/share/cassandra/apache-cassandra-2.0.1.jar:/usr/share/cassandra/apache-cassandra-thrift-2.0.1.jar:/usr/share/cassandra/apache-cassandra.jar:/usr/share/cassandra/stress.jar-XX:HeapDumpPath=/var/lib/cassandra/java_1380743391.hprof-XX:ErrorFile=/var/lib/cassandra/hs_err_1380743391.logorg.apache.cassandra.service.CassandraDaemon3


was (Author: sdelmas):
The issue seems to be that /var/run/cassandra is not being created during 
installation. So no pid file to use for stop. Interestingly enough the status 
call still fails to match the regular expression being used. I suspect that 
cmdline has changed compared to when jsvc was still used.

 /etc/init.d/cassandra stop and service don't work
 -

 Key: CASSANDRA-6116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6116
 Project: Cassandra
  Issue Type: Bug
  Components: Packaging
Reporter: Cathy Daw
Assignee: Eric Evans
Priority: Minor

 These use to work in 2.0.0 and appears to be introduced in 2.0.1
 Test Scenario
 {noformat}
 # Start Server
 automaton@ip-10-171-39-230:~$ sudo service cassandra start
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
 # Check Status
 automaton@ip-10-171-39-230:~$ nodetool status
 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  AddressLoad   Tokens  Owns   Host ID  
  Rack
 UN  127.0.0.1  81.72 KB   256 100.0%  
 e40ef77c-9cf7-4e27-b651-ede3b7269019  rack1
 # Check Status of service
 automaton@ip-10-171-39-230:~$ sudo service cassandra status
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
  * Cassandra is not running
 # Stop Server
 automaton@ip-10-171-39-230:~$ sudo service cassandra stop
 xss =  -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.5.jar 
 -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1862M -Xmx1862M 
 -Xmn200M -XX:+HeapDumpOnOutOfMemoryError -Xss256k
 # Verify Server is no longer up
 automaton@ip-10-171-39-230:~$ nodetool status
 

[2/4] git commit: fix merge conflict

2013-10-02 Thread jbellis
fix merge conflict


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/76cb10ca
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/76cb10ca
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/76cb10ca

Branch: refs/heads/cassandra-2.0
Commit: 76cb10ca92d46ccf9bb3f81f0ad897c6fb9bbf53
Parents: 70239e1
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 16:10:47 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 16:10:47 2013 -0500

--
 src/java/org/apache/cassandra/tracing/Tracing.java | 9 -
 1 file changed, 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76cb10ca/src/java/org/apache/cassandra/tracing/Tracing.java
--
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java 
b/src/java/org/apache/cassandra/tracing/Tracing.java
index aef3816..c000af9 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -170,16 +170,7 @@ public class Tracing
 CFMetaData cfMeta = CFMetaData.TraceSessionsCf;
 ColumnFamily cf = 
ArrayBackedSortedColumns.factory.create(cfMeta);
 addColumn(cf, buildName(cfMeta, bytes(duration)), 
elapsed);
- HEAD
-RowMutation mutation = new RowMutation(TRACE_KS, 
sessionIdBytes, cf);
-StorageProxy.mutate(Arrays.asList(mutation), 
ConsistencyLevel.ANY);
-||| merged common ancestors
-RowMutation mutation = new RowMutation(TRACE_KS, 
sessionIdBytes);
-mutation.add(cf);
-StorageProxy.mutate(Arrays.asList(mutation), 
ConsistencyLevel.ANY);
-===
 mutateWithCatch(new RowMutation(TRACE_KS, sessionIdBytes, 
cf));
- cassandra-1.2
 }
 });
 



[1/4] git commit: 6132

2013-10-02 Thread jbellis
Updated Branches:
  refs/heads/cassandra-1.2 64890d86d - 5440a0a67
  refs/heads/cassandra-2.0 70239e17b - 76cb10ca9
  refs/heads/trunk dca300d6c - 87e19fc8f


6132


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5440a0a6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5440a0a6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5440a0a6

Branch: refs/heads/cassandra-1.2
Commit: 5440a0a6767544d6ea1ba34f5d2a3e223f260fb5
Parents: 64890d8
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 14:09:29 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 14:09:29 2013 -0500

--
 .../org/apache/cassandra/net/CallbackInfo.java  | 17 ++--
 .../apache/cassandra/net/MessagingService.java  | 20 ---
 .../apache/cassandra/net/WriteCallbackInfo.java | 26 +++
 .../apache/cassandra/service/StorageProxy.java  | 27 
 4 files changed, 61 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5440a0a6/src/java/org/apache/cassandra/net/CallbackInfo.java
--
diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java 
b/src/java/org/apache/cassandra/net/CallbackInfo.java
index f0e48e9..f90df8d 100644
--- a/src/java/org/apache/cassandra/net/CallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/CallbackInfo.java
@@ -31,7 +31,6 @@ public class CallbackInfo
 {
 protected final InetAddress target;
 protected final IMessageCallback callback;
-protected final MessageOut? sentMessage;
 protected final IVersionedSerializer? serializer;
 
 /**
@@ -41,27 +40,15 @@ public class CallbackInfo
  * @param callback
  * @param serializer serializer to deserialize response message
  */
-public CallbackInfo(InetAddress target, IMessageCallback callback, 
IVersionedSerializer? serializer)
-{
-this(target, callback, null, serializer);
-}
-
-public CallbackInfo(InetAddress target, IMessageCallback callback, 
MessageOut? sentMessage, IVersionedSerializer? serializer)
+   public CallbackInfo(InetAddress target, IMessageCallback callback, 
IVersionedSerializer? serializer)
 {
 this.target = target;
 this.callback = callback;
-this.sentMessage = sentMessage;
 this.serializer = serializer;
 }
 
-/**
- * @return TRUE iff a hint should be written for this target.
- *
- * NOTE:
- * Assumes it is only called after the write of sentMessage to target 
has timed out.
- */
 public boolean shouldHint()
 {
-return sentMessage != null  StorageProxy.shouldHint(target);
+return false;
 }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5440a0a6/src/java/org/apache/cassandra/net/MessagingService.java
--
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java 
b/src/java/org/apache/cassandra/net/MessagingService.java
index a199e83..dd02ca6 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -329,8 +329,7 @@ public final class MessagingService implements 
MessagingServiceMBean
 
 if (expiredCallbackInfo.shouldHint())
 {
-assert expiredCallbackInfo.sentMessage != null;
-RowMutation rm = (RowMutation) 
expiredCallbackInfo.sentMessage.payload;
+RowMutation rm = (RowMutation) ((WriteCallbackInfo) 
expiredCallbackInfo).sentMessage.payload;
 return StorageProxy.submitHint(rm, 
expiredCallbackInfo.target, null, null);
 }
 
@@ -522,15 +521,18 @@ public final class MessagingService implements 
MessagingServiceMBean
 
 public String addCallback(IMessageCallback cb, MessageOut message, 
InetAddress to, long timeout)
 {
+assert message.verb != Verb.MUTATION; // mutations need to call the 
overload with a ConsistencyLevel
 String messageId = nextId();
-CallbackInfo previous;
-
-// If HH is enabled and this is a mutation message = store the 
message to track for potential hints.
-if (DatabaseDescriptor.hintedHandoffEnabled()  message.verb == 
Verb.MUTATION)
-previous = callbacks.put(messageId, new CallbackInfo(to, cb, 
message, callbackDeserializers.get(message.verb)), timeout);
-else
-previous = callbacks.put(messageId, new CallbackInfo(to, cb, 
callbackDeserializers.get(message.verb)), timeout);
+CallbackInfo previous = callbacks.put(messageId, new CallbackInfo(to, 
cb, callbackDeserializers.get(message.verb)), timeout);
+

[4/4] git commit: Merge branch 'cassandra-2.0' into trunk

2013-10-02 Thread jbellis
Merge branch 'cassandra-2.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/87e19fc8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/87e19fc8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/87e19fc8

Branch: refs/heads/trunk
Commit: 87e19fc8f56e4c5f90be1ca5c5f5037ea50a56c5
Parents: dca300d 76cb10c
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 16:10:57 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 16:10:57 2013 -0500

--
 src/java/org/apache/cassandra/tracing/Tracing.java | 9 -
 1 file changed, 9 deletions(-)
--




[3/4] git commit: fix merge conflict

2013-10-02 Thread jbellis
fix merge conflict


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/76cb10ca
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/76cb10ca
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/76cb10ca

Branch: refs/heads/trunk
Commit: 76cb10ca92d46ccf9bb3f81f0ad897c6fb9bbf53
Parents: 70239e1
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 16:10:47 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 16:10:47 2013 -0500

--
 src/java/org/apache/cassandra/tracing/Tracing.java | 9 -
 1 file changed, 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76cb10ca/src/java/org/apache/cassandra/tracing/Tracing.java
--
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java 
b/src/java/org/apache/cassandra/tracing/Tracing.java
index aef3816..c000af9 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -170,16 +170,7 @@ public class Tracing
 CFMetaData cfMeta = CFMetaData.TraceSessionsCf;
 ColumnFamily cf = 
ArrayBackedSortedColumns.factory.create(cfMeta);
 addColumn(cf, buildName(cfMeta, bytes(duration)), 
elapsed);
- HEAD
-RowMutation mutation = new RowMutation(TRACE_KS, 
sessionIdBytes, cf);
-StorageProxy.mutate(Arrays.asList(mutation), 
ConsistencyLevel.ANY);
-||| merged common ancestors
-RowMutation mutation = new RowMutation(TRACE_KS, 
sessionIdBytes);
-mutation.add(cf);
-StorageProxy.mutate(Arrays.asList(mutation), 
ConsistencyLevel.ANY);
-===
 mutateWithCatch(new RowMutation(TRACE_KS, sessionIdBytes, 
cf));
- cassandra-1.2
 }
 });
 



[jira] [Created] (CASSANDRA-6137) CQL3 SELECT IN CLAUSE inconsistent

2013-10-02 Thread Constance Eustace (JIRA)
Constance Eustace created CASSANDRA-6137:


 Summary: CQL3 SELECT IN CLAUSE inconsistent
 Key: CASSANDRA-6137
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6137
 Project: Cassandra
  Issue Type: Bug
 Environment: Ubuntu AWS Cassandra 2.0.1
Reporter: Constance Eustace
 Fix For: 2.0.1



We are encountering inconsistent results from CQL3 queries with column keys 
using IN clause in WHERE. This has been reproduced in cqlsh.

Rowkey is e_entid
Column key is p_prop

This returns roughly 21 rows for 21 column keys that match p_prop.

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB';

These three queries each return one row for the requested single column key in 
the IN clause:

SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:complete:count');
SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:all:count');
SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:fail:count');

This query returns ONLY ONE ROW (one column key), not three as I would expect 
from the three-column-key IN clause:

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:complete:count','urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');

This query does return two rows however for the requested two column keys:

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in (

'urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');





cqlsh describe table internal_submission.entity_job;

CREATE TABLE entity_job (
  e_entid text,
  p_prop text,
  describes text,
  dndcondition text,
  e_entlinks text,
  e_entname text,
  e_enttype text,
  ingeststatus text,
  ingeststatusdetail text,
  p_flags text,
  p_propid text,
  p_proplinks text,
  p_storage text,
  p_subents text,
  p_val text,
  p_vallang text,
  p_vallinks text,
  p_valtype text,
  p_valunit text,
  p_vars text,
  partnerid text,
  referenceid text,
  size int,
  sourceip text,
  submitdate bigint,
  submitevent text,
  userid text,
  version text,
  PRIMARY KEY (e_entid, p_prop)
) WITH
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=864000 AND
  index_interval=128 AND
  read_repair_chance=0.10 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};

CREATE INDEX internal_submission__JobDescribesIDX ON entity_job (describes);

CREATE INDEX internal_submission__JobDNDConditionIDX ON entity_job 
(dndcondition);

CREATE INDEX internal_submission__JobIngestStatusIDX ON entity_job 
(ingeststatus);

CREATE INDEX internal_submission__JobIngestStatusDetailIDX ON entity_job 
(ingeststatusdetail);

CREATE INDEX internal_submission__JobReferenceIDIDX ON entity_job (referenceid);

CREATE INDEX internal_submission__JobUserIDX ON entity_job (userid);

CREATE INDEX internal_submission__JobVersionIDX ON entity_job (version);

---

My suspicion is that the three-column-key IN Clause is translated (improperly 
or not) to a two-column key range with the assumption that the third column key 
is present in that range, but it isn't...




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


[jira] [Commented] (CASSANDRA-6137) CQL3 SELECT IN CLAUSE inconsistent

2013-10-02 Thread Constance Eustace (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6137?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784484#comment-13784484
 ] 

Constance Eustace commented on CASSANDRA-6137:
--

This is inconsistent/varies across rows in the table. 

 CQL3 SELECT IN CLAUSE inconsistent
 --

 Key: CASSANDRA-6137
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6137
 Project: Cassandra
  Issue Type: Bug
 Environment: Ubuntu AWS Cassandra 2.0.1
Reporter: Constance Eustace
 Fix For: 2.0.1


 We are encountering inconsistent results from CQL3 queries with column keys 
 using IN clause in WHERE. This has been reproduced in cqlsh.
 Rowkey is e_entid
 Column key is p_prop
 This returns roughly 21 rows for 21 column keys that match p_prop.
 cqlsh SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB';
 These three queries each return one row for the requested single column key 
 in the IN clause:
 SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
 ('urn:bby:pcm:job:ingest:content:complete:count');
 SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
 ('urn:bby:pcm:job:ingest:content:all:count');
 SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
 ('urn:bby:pcm:job:ingest:content:fail:count');
 This query returns ONLY ONE ROW (one column key), not three as I would expect 
 from the three-column-key IN clause:
 cqlsh SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
 ('urn:bby:pcm:job:ingest:content:complete:count','urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');
 This query does return two rows however for the requested two column keys:
 cqlsh SELECT 
 e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
  FROM internal_submission.Entity_Job WHERE e_entid = 
 '845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in (  
   
 'urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');
 cqlsh describe table internal_submission.entity_job;
 CREATE TABLE entity_job (
   e_entid text,
   p_prop text,
   describes text,
   dndcondition text,
   e_entlinks text,
   e_entname text,
   e_enttype text,
   ingeststatus text,
   ingeststatusdetail text,
   p_flags text,
   p_propid text,
   p_proplinks text,
   p_storage text,
   p_subents text,
   p_val text,
   p_vallang text,
   p_vallinks text,
   p_valtype text,
   p_valunit text,
   p_vars text,
   partnerid text,
   referenceid text,
   size int,
   sourceip text,
   submitdate bigint,
   submitevent text,
   userid text,
   version text,
   PRIMARY KEY (e_entid, p_prop)
 ) WITH
   bloom_filter_fp_chance=0.01 AND
   caching='KEYS_ONLY' AND
   comment='' AND
   dclocal_read_repair_chance=0.00 AND
   gc_grace_seconds=864000 AND
   index_interval=128 AND
   read_repair_chance=0.10 AND
   replicate_on_write='true' AND
   populate_io_cache_on_flush='false' AND
   default_time_to_live=0 AND
   speculative_retry='NONE' AND
   memtable_flush_period_in_ms=0 AND
   compaction={'class': 'SizeTieredCompactionStrategy'} AND
   compression={'sstable_compression': 'LZ4Compressor'};
 CREATE INDEX internal_submission__JobDescribesIDX ON entity_job (describes);
 CREATE INDEX internal_submission__JobDNDConditionIDX ON entity_job 
 (dndcondition);
 CREATE INDEX internal_submission__JobIngestStatusIDX ON entity_job 
 (ingeststatus);
 CREATE INDEX internal_submission__JobIngestStatusDetailIDX ON entity_job 
 (ingeststatusdetail);
 CREATE INDEX internal_submission__JobReferenceIDIDX ON entity_job 
 (referenceid);
 CREATE INDEX internal_submission__JobUserIDX ON entity_job (userid);
 CREATE INDEX internal_submission__JobVersionIDX ON entity_job (version);
 ---
 My suspicion is that the three-column-key IN Clause is translated (improperly 
 or not) to a two-column key range with the assumption that the third 

[jira] [Updated] (CASSANDRA-6137) CQL3 SELECT IN CLAUSE inconsistent

2013-10-02 Thread Constance Eustace (JIRA)

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

Constance Eustace updated CASSANDRA-6137:
-

Component/s: Core
Description: 
We are encountering inconsistent results from CQL3 queries with column keys 
using IN clause in WHERE. This has been reproduced in cqlsh.

Rowkey is e_entid
Column key is p_prop

This returns roughly 21 rows for 21 column keys that match p_prop.

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB';

These three queries each return one row for the requested single column key in 
the IN clause:

SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:complete:count');
SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:all:count');
SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:fail:count');

This query returns ONLY ONE ROW (one column key), not three as I would expect 
from the three-column-key IN clause:

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in 
('urn:bby:pcm:job:ingest:content:complete:count','urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');

This query does return two rows however for the requested two column keys:

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 
'845b38f1-2b91-11e3-854d-126aad0075d4-CJOB'  AND p_prop in (

'urn:bby:pcm:job:ingest:content:all:count','urn:bby:pcm:job:ingest:content:fail:count');





cqlsh describe table internal_submission.entity_job;

CREATE TABLE entity_job (
  e_entid text,
  p_prop text,
  describes text,
  dndcondition text,
  e_entlinks text,
  e_entname text,
  e_enttype text,
  ingeststatus text,
  ingeststatusdetail text,
  p_flags text,
  p_propid text,
  p_proplinks text,
  p_storage text,
  p_subents text,
  p_val text,
  p_vallang text,
  p_vallinks text,
  p_valtype text,
  p_valunit text,
  p_vars text,
  partnerid text,
  referenceid text,
  size int,
  sourceip text,
  submitdate bigint,
  submitevent text,
  userid text,
  version text,
  PRIMARY KEY (e_entid, p_prop)
) WITH
  bloom_filter_fp_chance=0.01 AND
  caching='KEYS_ONLY' AND
  comment='' AND
  dclocal_read_repair_chance=0.00 AND
  gc_grace_seconds=864000 AND
  index_interval=128 AND
  read_repair_chance=0.10 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};

CREATE INDEX internal_submission__JobDescribesIDX ON entity_job (describes);

CREATE INDEX internal_submission__JobDNDConditionIDX ON entity_job 
(dndcondition);

CREATE INDEX internal_submission__JobIngestStatusIDX ON entity_job 
(ingeststatus);

CREATE INDEX internal_submission__JobIngestStatusDetailIDX ON entity_job 
(ingeststatusdetail);

CREATE INDEX internal_submission__JobReferenceIDIDX ON entity_job (referenceid);

CREATE INDEX internal_submission__JobUserIDX ON entity_job (userid);

CREATE INDEX internal_submission__JobVersionIDX ON entity_job (version);

---

My suspicion is that the three-column-key IN Clause is translated (improperly 
or not) to a two-column key range with the assumption that the third column key 
is present in that range, but it isn't...


  was:

We are encountering inconsistent results from CQL3 queries with column keys 
using IN clause in WHERE. This has been reproduced in cqlsh.

Rowkey is e_entid
Column key is p_prop

This returns roughly 21 rows for 21 column keys that match p_prop.

cqlsh SELECT 
e_entid,e_entname,e_enttype,p_prop,p_flags,p_propid,e_entlinks,p_proplinks,p_subents,p_val,p_vallinks,p_vars
 FROM internal_submission.Entity_Job WHERE e_entid = 

[jira] [Updated] (CASSANDRA-6107) CQL3 Batch statement memory leak

2013-10-02 Thread Lyuben Todorov (JIRA)

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

Lyuben Todorov updated CASSANDRA-6107:
--

Attachment: 6107.patch

 CQL3 Batch statement memory leak
 

 Key: CASSANDRA-6107
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6107
 Project: Cassandra
  Issue Type: Bug
  Components: API, Core
 Environment: - CASS version: 1.2.8 or 2.0.1, same issue seen in both
 - Running on OSX MacbookPro
 - Sun JVM 1.7
 - Single local cassandra node
 - both CMS and G1 GC used
 - we are using the cass-JDBC driver to submit our batches
Reporter: Constance Eustace
Assignee: Lyuben Todorov
Priority: Minor
 Fix For: 1.2.11

 Attachments: 6107.patch


 We are doing large volume insert/update tests on a CASS via CQL3. 
 Using 4GB heap, after roughly 750,000 updates create/update 75,000 row keys, 
 we run out of heap, and it never dissipates, and we begin getting this 
 infamous error which many people seem to be encountering:
 WARN [ScheduledTasks:1] 2013-09-26 16:17:10,752 GCInspector.java (line 142) 
 Heap is 0.9383457210434385 full.  You may need to reduce memtable and/or 
 cache sizes.  Cassandra will now flush up to the two largest memtables to 
 free up memory.  Adjust flush_largest_memtables_at threshold in 
 cassandra.yaml if you don't want Cassandra to do this automatically
  INFO [ScheduledTasks:1] 2013-09-26 16:17:10,753 StorageService.java (line 
 3614) Unable to reduce heap usage since there are no dirty column families
 8 and 12 GB heaps appear to delay the problem by roughly proportionate 
 amounts of 75,000 - 100,000 rowkeys per 4GB. Each run of 50,000 row key 
 creations sees the heap grow and never shrink again. 
 We have attempted to no effect:
 - removing all secondary indexes to see if that alleviates overuse of bloom 
 filters 
 - adjusted parameters for compaction throughput
 - adjusted memtable flush thresholds and other parameters 
 By examining heapdumps, it seems apparent that the problem is perpetual 
 retention of CQL3 BATCH statements. We have even tried dropping the keyspaces 
 after the updates and the CQL3 statement are still visible in the heapdump, 
 and after many many many CMS GC runs. G1 also showed this issue.
 The 750,000 statements are broken into batches of roughly 200 statements.



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


[jira] [Commented] (CASSANDRA-6132) CL.ANY writes can still time out

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784630#comment-13784630
 ] 

Aleksey Yeschenko commented on CASSANDRA-6132:
--

+1

 CL.ANY writes can still time out
 

 Key: CASSANDRA-6132
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6132
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 1.2.11, 2.0.2

 Attachments: 6132.txt, 6132-v2.txt


 If we know that all replicas are down at the beginning of a mutation, we will 
 write a hint and return success.
 But if we do not, we will attemp to write to replicas, time out, return 
 failure, and then write a hint, violating our contract that (unless the 
 coordinator goes down), writes at CL.ANY should always succeed.



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


[jira] [Commented] (CASSANDRA-6107) CQL3 Batch statement memory leak

2013-10-02 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784635#comment-13784635
 ] 

Aleksey Yeschenko commented on CASSANDRA-6107:
--

I don't think the issue here is (just) large individual prepared statements. 
It's the total size that all the prepared statements are occupying. That's what 
should be tracked and limited, not just the individual ones.

 CQL3 Batch statement memory leak
 

 Key: CASSANDRA-6107
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6107
 Project: Cassandra
  Issue Type: Bug
  Components: API, Core
 Environment: - CASS version: 1.2.8 or 2.0.1, same issue seen in both
 - Running on OSX MacbookPro
 - Sun JVM 1.7
 - Single local cassandra node
 - both CMS and G1 GC used
 - we are using the cass-JDBC driver to submit our batches
Reporter: Constance Eustace
Assignee: Lyuben Todorov
Priority: Minor
 Fix For: 1.2.11

 Attachments: 6107.patch


 We are doing large volume insert/update tests on a CASS via CQL3. 
 Using 4GB heap, after roughly 750,000 updates create/update 75,000 row keys, 
 we run out of heap, and it never dissipates, and we begin getting this 
 infamous error which many people seem to be encountering:
 WARN [ScheduledTasks:1] 2013-09-26 16:17:10,752 GCInspector.java (line 142) 
 Heap is 0.9383457210434385 full.  You may need to reduce memtable and/or 
 cache sizes.  Cassandra will now flush up to the two largest memtables to 
 free up memory.  Adjust flush_largest_memtables_at threshold in 
 cassandra.yaml if you don't want Cassandra to do this automatically
  INFO [ScheduledTasks:1] 2013-09-26 16:17:10,753 StorageService.java (line 
 3614) Unable to reduce heap usage since there are no dirty column families
 8 and 12 GB heaps appear to delay the problem by roughly proportionate 
 amounts of 75,000 - 100,000 rowkeys per 4GB. Each run of 50,000 row key 
 creations sees the heap grow and never shrink again. 
 We have attempted to no effect:
 - removing all secondary indexes to see if that alleviates overuse of bloom 
 filters 
 - adjusted parameters for compaction throughput
 - adjusted memtable flush thresholds and other parameters 
 By examining heapdumps, it seems apparent that the problem is perpetual 
 retention of CQL3 BATCH statements. We have even tried dropping the keyspaces 
 after the updates and the CQL3 statement are still visible in the heapdump, 
 and after many many many CMS GC runs. G1 also showed this issue.
 The 750,000 statements are broken into batches of roughly 200 statements.



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


[jira] [Commented] (CASSANDRA-6107) CQL3 Batch statement memory leak

2013-10-02 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13784658#comment-13784658
 ] 

Jonathan Ellis commented on CASSANDRA-6107:
---

Right.  Use the size you're calculating as the weight in the cache Map.

 CQL3 Batch statement memory leak
 

 Key: CASSANDRA-6107
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6107
 Project: Cassandra
  Issue Type: Bug
  Components: API, Core
 Environment: - CASS version: 1.2.8 or 2.0.1, same issue seen in both
 - Running on OSX MacbookPro
 - Sun JVM 1.7
 - Single local cassandra node
 - both CMS and G1 GC used
 - we are using the cass-JDBC driver to submit our batches
Reporter: Constance Eustace
Assignee: Lyuben Todorov
Priority: Minor
 Fix For: 1.2.11

 Attachments: 6107.patch


 We are doing large volume insert/update tests on a CASS via CQL3. 
 Using 4GB heap, after roughly 750,000 updates create/update 75,000 row keys, 
 we run out of heap, and it never dissipates, and we begin getting this 
 infamous error which many people seem to be encountering:
 WARN [ScheduledTasks:1] 2013-09-26 16:17:10,752 GCInspector.java (line 142) 
 Heap is 0.9383457210434385 full.  You may need to reduce memtable and/or 
 cache sizes.  Cassandra will now flush up to the two largest memtables to 
 free up memory.  Adjust flush_largest_memtables_at threshold in 
 cassandra.yaml if you don't want Cassandra to do this automatically
  INFO [ScheduledTasks:1] 2013-09-26 16:17:10,753 StorageService.java (line 
 3614) Unable to reduce heap usage since there are no dirty column families
 8 and 12 GB heaps appear to delay the problem by roughly proportionate 
 amounts of 75,000 - 100,000 rowkeys per 4GB. Each run of 50,000 row key 
 creations sees the heap grow and never shrink again. 
 We have attempted to no effect:
 - removing all secondary indexes to see if that alleviates overuse of bloom 
 filters 
 - adjusted parameters for compaction throughput
 - adjusted memtable flush thresholds and other parameters 
 By examining heapdumps, it seems apparent that the problem is perpetual 
 retention of CQL3 BATCH statements. We have even tried dropping the keyspaces 
 after the updates and the CQL3 statement are still visible in the heapdump, 
 and after many many many CMS GC runs. G1 also showed this issue.
 The 750,000 statements are broken into batches of roughly 200 statements.



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


[1/8] git commit: 6132

2013-10-02 Thread jbellis
Updated Branches:
  refs/heads/cassandra-1.2 5440a0a67 - 6b5874503
  refs/heads/cassandra-2.0 76cb10ca9 - e7c90e04c
  refs/heads/trunk 87e19fc8f - 70dc32023


6132


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5440a0a6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5440a0a6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5440a0a6

Branch: refs/heads/cassandra-2.0
Commit: 5440a0a6767544d6ea1ba34f5d2a3e223f260fb5
Parents: 64890d8
Author: Jonathan Ellis jbel...@apache.org
Authored: Wed Oct 2 14:09:29 2013 -0500
Committer: Jonathan Ellis jbel...@apache.org
Committed: Wed Oct 2 14:09:29 2013 -0500

--
 .../org/apache/cassandra/net/CallbackInfo.java  | 17 ++--
 .../apache/cassandra/net/MessagingService.java  | 20 ---
 .../apache/cassandra/net/WriteCallbackInfo.java | 26 +++
 .../apache/cassandra/service/StorageProxy.java  | 27 
 4 files changed, 61 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5440a0a6/src/java/org/apache/cassandra/net/CallbackInfo.java
--
diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java 
b/src/java/org/apache/cassandra/net/CallbackInfo.java
index f0e48e9..f90df8d 100644
--- a/src/java/org/apache/cassandra/net/CallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/CallbackInfo.java
@@ -31,7 +31,6 @@ public class CallbackInfo
 {
 protected final InetAddress target;
 protected final IMessageCallback callback;
-protected final MessageOut? sentMessage;
 protected final IVersionedSerializer? serializer;
 
 /**
@@ -41,27 +40,15 @@ public class CallbackInfo
  * @param callback
  * @param serializer serializer to deserialize response message
  */
-public CallbackInfo(InetAddress target, IMessageCallback callback, 
IVersionedSerializer? serializer)
-{
-this(target, callback, null, serializer);
-}
-
-public CallbackInfo(InetAddress target, IMessageCallback callback, 
MessageOut? sentMessage, IVersionedSerializer? serializer)
+   public CallbackInfo(InetAddress target, IMessageCallback callback, 
IVersionedSerializer? serializer)
 {
 this.target = target;
 this.callback = callback;
-this.sentMessage = sentMessage;
 this.serializer = serializer;
 }
 
-/**
- * @return TRUE iff a hint should be written for this target.
- *
- * NOTE:
- * Assumes it is only called after the write of sentMessage to target 
has timed out.
- */
 public boolean shouldHint()
 {
-return sentMessage != null  StorageProxy.shouldHint(target);
+return false;
 }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5440a0a6/src/java/org/apache/cassandra/net/MessagingService.java
--
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java 
b/src/java/org/apache/cassandra/net/MessagingService.java
index a199e83..dd02ca6 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -329,8 +329,7 @@ public final class MessagingService implements 
MessagingServiceMBean
 
 if (expiredCallbackInfo.shouldHint())
 {
-assert expiredCallbackInfo.sentMessage != null;
-RowMutation rm = (RowMutation) 
expiredCallbackInfo.sentMessage.payload;
+RowMutation rm = (RowMutation) ((WriteCallbackInfo) 
expiredCallbackInfo).sentMessage.payload;
 return StorageProxy.submitHint(rm, 
expiredCallbackInfo.target, null, null);
 }
 
@@ -522,15 +521,18 @@ public final class MessagingService implements 
MessagingServiceMBean
 
 public String addCallback(IMessageCallback cb, MessageOut message, 
InetAddress to, long timeout)
 {
+assert message.verb != Verb.MUTATION; // mutations need to call the 
overload with a ConsistencyLevel
 String messageId = nextId();
-CallbackInfo previous;
-
-// If HH is enabled and this is a mutation message = store the 
message to track for potential hints.
-if (DatabaseDescriptor.hintedHandoffEnabled()  message.verb == 
Verb.MUTATION)
-previous = callbacks.put(messageId, new CallbackInfo(to, cb, 
message, callbackDeserializers.get(message.verb)), timeout);
-else
-previous = callbacks.put(messageId, new CallbackInfo(to, cb, 
callbackDeserializers.get(message.verb)), timeout);
+CallbackInfo previous = callbacks.put(messageId, new CallbackInfo(to, 
cb, callbackDeserializers.get(message.verb)), timeout);
+

  1   2   >