[jira] [Issue Comment Edited] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen edited comment on CASSANDRA-2816 at 6/25/11 9:04 AM:


This sounds very interesting.

We have also spotted very noticable issues with full GCs when the merkle trees 
are passed around. Hopefully this could fix that too.

I will see if I can get this patch tested somewhere if it is ready for that.

On a side topic, given the importance of getting tombstones properly 
synchronized within GCGraceSeconds, would it be an potential interesting idea 
to separate tombstones in different sstables to reduce the need to scan the 
whole dataset very frequently in the first place?

Another thought may be to make compaction deterministic or synchronized by a 
master across nodes so for older data, all we needed was to compare pre-stored 
md5s of how whole sstables? 

That is, while keeping the masterless design for updates, we could consider a 
master based design for how older data is being organized by the compactor. so 
it would be much easier to verify that old data is the same without any large 
regular scans and that data is really the same after big compactions etc.


  was (Author: terjem):
Sounds good to me.

This sounds very interesting.
We have also spotted very noticable issues with full GCs when the merkle trees 
are passed around. Hopefully this could fix that too.

I will see if I can get this patch tested somewhere if it is ready for that.

On a side topic, given the importance of getting tombstones properly 
synchronized within GCGraceSeconds, would it be an potential interesting idea 
to separate tombstones in different sstables to reduce the need to scan the 
whole dataset very frequently in the first place?

Another thought may be to make compaction deterministic or synchronized by a 
master across nodes so for older data, all we needed was to compare pre-stored 
md5s of how whole sstables? 

That is, while keeping the masterless design for updates, we could consider a 
master based design for how older data is being organized by the compactor. so 
it would be much easier to verify that old data is the same without any large 
regular scans and that data is really the same after big compactions etc.

  
 Repair doesn't synchronize merkle tree creation properly
 

 Key: CASSANDRA-2816
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
  Labels: repair
 Fix For: 0.8.2

 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch


 Being a little slow, I just realized after having opened CASSANDRA-2811 and 
 CASSANDRA-2815 that there is a more general problem with repair.
 When a repair is started, it will send a number of merkle tree to its 
 neighbor as well as himself and assume for correction that the building of 
 those trees will be started on every node roughly at the same time (if not, 
 we end up comparing data snapshot at different time and will thus mistakenly 
 repair a lot of useless data). This is bogus for many reasons:
 * Because validation compaction runs on the same executor that other 
 compaction, the start of the validation on the different node is subject to 
 other compactions. 0.8 mitigates this in a way by being multi-threaded (and 
 thus there is less change to be blocked a long time by a long running 
 compaction), but the compaction executor being bounded, its still a problem)
 * if you run a nodetool repair without arguments, it will repair every CFs. 
 As a consequence it will generate lots of merkle tree requests and all of 
 those requests will be issued at the same time. Because even in 0.8 the 
 compaction executor is bounded, some of those validations will end up being 
 queued behind the first ones. Even assuming that the different validation are 
 submitted in the same order on each node (which isn't guaranteed either), 
 there is no guarantee that on all nodes, the first validation will take the 
 same time, hence desynchronizing the queued ones.
 Overall, it is important for the precision of repair that for a given CF and 
 range (which is the unit at which trees are computed), we make sure that all 
 node will start the validation at the same time (or, since we can't do magic, 
 as close as possible).
 One (reasonably simple) proposition to fix this would be to have repair 
 schedule validation compactions across nodes one by one (i.e, one CF/range at 
 a time), waiting for all nodes to return their tree before submitting the 
 next request. Then on each node, we should make sure that the node will 

[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2816:
---

Sounds good to me.

This sounds very interesting.
We have also spotted very noticable issues with full GCs when the merkle trees 
are passed around. Hopefully this could fix that too.

I will see if I can get this patch tested somewhere if it is ready for that.

On a side topic, given the importance of getting tombstones properly 
synchronized within GCGraceSeconds, would it be an potential interesting idea 
to separate tombstones in different sstables to reduce the need to scan the 
whole dataset very frequently in the first place?

Another thought may be to make compaction deterministic or synchronized by a 
master across nodes so for older data, all we needed was to compare pre-stored 
md5s of how whole sstables? 

That is, while keeping the masterless design for updates, we could consider a 
master based design for how older data is being organized by the compactor. so 
it would be much easier to verify that old data is the same without any large 
regular scans and that data is really the same after big compactions etc.


 Repair doesn't synchronize merkle tree creation properly
 

 Key: CASSANDRA-2816
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
  Labels: repair
 Fix For: 0.8.2

 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch


 Being a little slow, I just realized after having opened CASSANDRA-2811 and 
 CASSANDRA-2815 that there is a more general problem with repair.
 When a repair is started, it will send a number of merkle tree to its 
 neighbor as well as himself and assume for correction that the building of 
 those trees will be started on every node roughly at the same time (if not, 
 we end up comparing data snapshot at different time and will thus mistakenly 
 repair a lot of useless data). This is bogus for many reasons:
 * Because validation compaction runs on the same executor that other 
 compaction, the start of the validation on the different node is subject to 
 other compactions. 0.8 mitigates this in a way by being multi-threaded (and 
 thus there is less change to be blocked a long time by a long running 
 compaction), but the compaction executor being bounded, its still a problem)
 * if you run a nodetool repair without arguments, it will repair every CFs. 
 As a consequence it will generate lots of merkle tree requests and all of 
 those requests will be issued at the same time. Because even in 0.8 the 
 compaction executor is bounded, some of those validations will end up being 
 queued behind the first ones. Even assuming that the different validation are 
 submitted in the same order on each node (which isn't guaranteed either), 
 there is no guarantee that on all nodes, the first validation will take the 
 same time, hence desynchronizing the queued ones.
 Overall, it is important for the precision of repair that for a given CF and 
 range (which is the unit at which trees are computed), we make sure that all 
 node will start the validation at the same time (or, since we can't do magic, 
 as close as possible).
 One (reasonably simple) proposition to fix this would be to have repair 
 schedule validation compactions across nodes one by one (i.e, one CF/range at 
 a time), waiting for all nodes to return their tree before submitting the 
 next request. Then on each node, we should make sure that the node will start 
 the validation compaction as soon as requested. For that, we probably want to 
 have a specific executor for validation compaction and:
 * either we fail the whole repair whenever one node is not able to execute 
 the validation compaction right away (because no thread are available right 
 away).
 * we simply tell the user that if he start too many repairs in parallel, he 
 may start seeing some of those repairing more data than it should.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2521:
---

I have not tested Jonathan's updated version, but the version before seems to 
work good so far.

Repair still keep data around while running, but that may be solved with 
CASSANDRA-2816.

Other than that, sstables in general seems to disappear within a minute of 
being freed from compaction. 

Except for when running repair, I have to actively hunt for -Compacted files 
to find them now. 

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2521:
-

bq. Repair still keep data around while running, but that may be solved with 
CASSANDRA-2816.

This is the expected behavior. Or more precisely, it all depends on what we're 
talking about. When repair is working with a sstable, we cannot and we should 
not delete it. Even if they have been compacted, if we're in the middle of 
streaming them, we should not delete it, there is nothing we can do about it. 
Now, I've tried to make it so that as soon as a file is not needed by repair 
anymore, it is deleted and thus if what you're talking about is sstable that 
get kept around forever, then it is a bug in the patch. To have a -Compacted 
file around don't mean the sstable can be deleted and it will never be.

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2498) Improve read performance in update-intensive workload

2011-06-25 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2498:
---

bq. IF the queries are for most-recent-data

Actually...  I can't think of a good way for us to infer that (say) higher 
timeuuid comparators always correspond to higher column timestamps.

 Improve read performance in update-intensive workload
 -

 Key: CASSANDRA-2498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2498
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
  Labels: ponies
 Fix For: 1.0


 Read performance in an update-heavy environment relies heavily on compaction 
 to maintain good throughput. (This is not the case for workloads where rows 
 are only inserted once, because the bloom filter keeps us from having to 
 check sstables unnecessarily.)
 Very early versions of Cassandra attempted to mitigate this by checking 
 sstables in descending generation order (mostly equivalent to descending 
 mtime): once all the requested columns were found, it would not check any 
 older sstables.
 This was incorrect, because data timestamp will not correspond to sstable 
 timestamp, both because compaction has the side effect of refreshing data 
 to a newer sstable, and because hintead handoff may send us data older than 
 what we already have.
 Instead, we could create a per-sstable piece of metadata containing the most 
 recent (client-specified) timestamp for any column in the sstable.  We could 
 then sort sstables by this timestamp instead, and perform a similar 
 optimization (if the remaining sstable client-timestamps are older than the 
 oldest column found in the desired result set so far, we don't need to look 
 further). Since under almost every workload, client timestamps of data in a 
 given sstable will tend to be similar, we expect this to cut the number of 
 sstables down proportionally to how frequently each column in the row is 
 updated. (If each column is updated with each write, we only have to check a 
 single sstable.)
 This may also be useful information when deciding which SSTables to compact.
 (Note that this optimization is only appropriate for named-column queries, 
 not slice queries, since we don't know what non-overlapping columns may exist 
 in older sstables.)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2521:
---

Yes, no problem that files stay for a while during repair, but I believe 
currently, files are not deleted until the entire repair is finished even if 
they have completed streaming, and I suppose that is not needed?

Regardless, I believe CASSANDRA-2816 may fix this?

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2521:
---

When that is said, I have to actually verify that nothing is deleted until the 
repair is completed. I am pretty sure this happened with GC doing the cleanup, 
but I have actually not verified that with this patch.

I just observe that I do not see anything get deleted until it finishes, but 
just because I don't see that with some occasional ls's does not mean it does 
not happen.

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2826) Debian Package for 0.8 is missing

2011-06-25 Thread paul cannon (JIRA)

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

paul cannon updated CASSANDRA-2826:
---

Assignee: Eric Evans  (was: paul cannon)

I think Eric has to take care of this.

 Debian Package for 0.8 is missing
 -

 Key: CASSANDRA-2826
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2826
 Project: Cassandra
  Issue Type: Bug
  Components: Packaging
Affects Versions: 0.8.0
Reporter: Scott Mitchell
Assignee: Eric Evans
Priority: Minor
 Fix For: 0.8.2


 In file
 http://www.apache.org/dist/cassandra/debian/dists/08x/InRelease
 Codename: sid
 Should be changed to
 Codename: 08x

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2521:
---

Hm... there may actually be something here.
I am getting some *Compacted which does not seem to go away, not even after 7 
hours after finished.

3 Compactions 2 of them have left 1 sstable, one has left 2.

I also happened to run 3 repairs, but one compaction starts just after one of 
those repairs finished (the other 2 starts and finished during the repair.

This is running 2 as minimum compaction threshold. 
The compaction which left 2 files, left them after a compaction of 3 files.
The 2 other *Compacted was made after  compactions of 2 files

The system is under reasonably heavy input of data and sstables are added, 
compacted and removed all the time. Just these which do not get removed and the 
oldest is 8 hours now.

Note, I do not use mmap.

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable

2011-06-25 Thread Terje Marthinussen (JIRA)

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

Terje Marthinussen commented on CASSANDRA-2521:
---

Hm, forget that. All 3 compactions happens during the repairs.

This test is also using CASSANDRA-2433 and there are no errors reported during 
the repairs.

Only one of the leftover sstables is listed in the log entries for stream 
context metadata

 Move away from Phantom References for Compaction/Memtable
 -

 Key: CASSANDRA-2521
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Sylvain Lebresne
 Fix For: 1.0

 Attachments: 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 
 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 
 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt


 http://wiki.apache.org/cassandra/MemtableSSTable
 Let's move to using reference counting instead of relying on GC to be called 
 in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (CASSANDRA-2827) Thrift error

2011-06-25 Thread Olivier Smadja (JIRA)
Thrift error


 Key: CASSANDRA-2827
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2827
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.4
 Environment: 2 nodes with 0.7.4 on linux
Reporter: Olivier Smadja


This exeception occured of a non seed node.

ERROR [pool-1-thread-9] 2011-06-25 17:41:37,723 CustomTThreadPoolServer.java 
(line 218) Thrift error occurred during processing of message.
org.apache.thrift.TException: Negative length: -2147418111
at 
org.apache.thrift.protocol.TBinaryProtocol.checkReadLength(TBinaryProtocol.java:388)
at 
org.apache.thrift.protocol.TBinaryProtocol.readBinary(TBinaryProtocol.java:363)
at 
org.apache.cassandra.thrift.Cassandra$batch_mutate_args.read(Cassandra.java:15964)
at 
org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3023)
at 
org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
at 
org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
at java.lang.Thread.run(Thread.java:619)


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2827) Thrift error

2011-06-25 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2827:
---

Are you ensuring that only a single thread uses each Thrift connection at a 
time?

 Thrift error
 

 Key: CASSANDRA-2827
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2827
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.4
 Environment: 2 nodes with 0.7.4 on linux
Reporter: Olivier Smadja

 This exeception occured of a non seed node.
 ERROR [pool-1-thread-9] 2011-06-25 17:41:37,723 CustomTThreadPoolServer.java 
 (line 218) Thrift error occurred during processing of message.
 org.apache.thrift.TException: Negative length: -2147418111
   at 
 org.apache.thrift.protocol.TBinaryProtocol.checkReadLength(TBinaryProtocol.java:388)
   at 
 org.apache.thrift.protocol.TBinaryProtocol.readBinary(TBinaryProtocol.java:363)
   at 
 org.apache.cassandra.thrift.Cassandra$batch_mutate_args.read(Cassandra.java:15964)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3023)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
   at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
   at java.lang.Thread.run(Thread.java:619)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2827) Thrift error

2011-06-25 Thread Olivier Smadja (JIRA)

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

Olivier Smadja commented on CASSANDRA-2827:
---

Well,

We don't use Thrift directly, we are using Hector to access cassandra.

Is there any issue using Hector with multi-threaded app ?

Thanks,
Olivier 

 Thrift error
 

 Key: CASSANDRA-2827
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2827
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.4
 Environment: 2 nodes with 0.7.4 on linux
Reporter: Olivier Smadja

 This exeception occured of a non seed node.
 ERROR [pool-1-thread-9] 2011-06-25 17:41:37,723 CustomTThreadPoolServer.java 
 (line 218) Thrift error occurred during processing of message.
 org.apache.thrift.TException: Negative length: -2147418111
   at 
 org.apache.thrift.protocol.TBinaryProtocol.checkReadLength(TBinaryProtocol.java:388)
   at 
 org.apache.thrift.protocol.TBinaryProtocol.readBinary(TBinaryProtocol.java:363)
   at 
 org.apache.cassandra.thrift.Cassandra$batch_mutate_args.read(Cassandra.java:15964)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3023)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
   at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
   at java.lang.Thread.run(Thread.java:619)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2827) Thrift error

2011-06-25 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2827:
---

Not if you are using the pooling correctly.

 Thrift error
 

 Key: CASSANDRA-2827
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2827
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.4
 Environment: 2 nodes with 0.7.4 on linux
Reporter: Olivier Smadja

 This exeception occured of a non seed node.
 ERROR [pool-1-thread-9] 2011-06-25 17:41:37,723 CustomTThreadPoolServer.java 
 (line 218) Thrift error occurred during processing of message.
 org.apache.thrift.TException: Negative length: -2147418111
   at 
 org.apache.thrift.protocol.TBinaryProtocol.checkReadLength(TBinaryProtocol.java:388)
   at 
 org.apache.thrift.protocol.TBinaryProtocol.readBinary(TBinaryProtocol.java:363)
   at 
 org.apache.cassandra.thrift.Cassandra$batch_mutate_args.read(Cassandra.java:15964)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3023)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
   at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
   at java.lang.Thread.run(Thread.java:619)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira