[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2015-04-28 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-7275 at 4/28/15 4:46 PM:
--

Since this has come up again a few times, I propose that in 3.X we deliver 
CASSANDRA-8496, since this would prevent threads being locked up - it would 
only leave CommitLog records to replay and memtable space unreclaimed, which 
would be a big improvement from the status quo at least. It is by itself not 
super challenging, and relatively safe. Deciding how we safely recover _fully_ 
is a more challenging question.


was (Author: benedict):
Since this has come up again a few times, I propose tha

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 2.0.15

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-12-21 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-7275 at 12/21/14 11:14 AM:


I think somebody with better knowledge of the bookkeeping tables needs to chime 
in here, to give an opinion on if we can safely do this to any we care about.

We still need to decide what to do about the non-whitelisted tables, though. 
They most likely want to not mark the CL clean at least for their affected 
segments, but possibly (esp. for local-only) indefinitely for the affected 
table, until reboot, to guarantee no data bugs.


was (Author: benedict):
I think somebody with better knowledge of the bookkeeping tables needs to chime 
in here, to give an opinion on if we can safely do this to any we care about.

We still need to decide what to do about the non-whitelisted tables, though. 
They most likely want to not mark the CL clean at least for their affected 
segments, but possibly indefinitely for the affected table, until reboot, to 
guarantee no data bugs.

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 2.0.12

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-12-17 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-7275 at 12/17/14 11:47 AM:


I've filed CASSANDRA-8496, which would help with this problem in 2.1 only. It 
isn't sufficient to ensure the server stays stable, but would both avoid 
forward progress being stopped by errors on the post flusher, and that the 
affected commit log records would be retained indefinitely without resulting in 
infinite commit log growth. I've also filed CASSANDRA-8497 and CASSANDRA-8498 
which should help avoid data corruption in the cluster.


was (Author: benedict):
I've filed CASSANDRA-8496, which would help with this problem in 2.1 only. It 
isn't sufficient to ensure the server stays stable, but would both avoid 
forward progress being stopped by errors on the post flusher, and that the 
affected commit log records would be retained indefinitely without resulting in 
infinite commit log growth.

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 2.0.12

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-12-16 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis edited comment on CASSANDRA-7275 at 12/16/14 10:41 PM:
--

bq. consider there could be a programming error in the flush of the memtable or 
replacing flushed one

I don't know that hand waving about potential bugs gets us anywhere.  There 
could be programming errors anywhere, including in mark the segment flushed 
when it wasn't panic mode.  The right solution to bugs is QA, not hoping that 
you can guess where unexpected exceptions will happen and provide a safety net.

Edit: which is to say that I think the right scope here is, what do we do if 
we can't flush because of an i/o error, which is an expected condition that C* 
should be able to cope with cleanly.


was (Author: jbellis):
bq. consider there could be a programming error in the flush of the memtable or 
replacing flushed one

I don't know that hand waving about potential bugs gets us anywhere.  There 
could be programming errors anywhere, including in mark the segment flushed 
when it wasn't panic mode.  The right solution to bugs is QA, not hoping that 
you can guess where unexpected exceptions will happen and provide a safety net.

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 2.0.12

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-12-16 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich edited comment on CASSANDRA-7275 at 12/16/14 11:50 PM:
---

The problem is that there is no way to tell if hard-link problem is a actual 
fs/disk problem or programming error, right now it looks like a programming 
error because it snapshot tries to create duplicate hard-link to the same file 
as I mentioned in the CASSANDRA-8476 so if there is no way to tell how 
reasonable is it to enforce shutdown or any rule from disk_failure_policy?

bq. If it's our bug, then you may need a temporary patch while we figure out 
the cause, but I still don't think that kind of // this shouldn't happen code 
should be shipped officially.

If it's your problem it's my problem as well, we have a work-around for now (as 
I guess most of the people do) but my intention in this ticket to fix this 
problem for good instead of just fixing the symptom of it (being aforementioned 
duplicate hard-link problem).


was (Author: xedin):
The problem is that there is no way to tell if hard-link problem is a actual 
fs/disk problem or programming error, right now it looks like a programming 
error because it snapshot tries to create duplicate hard-link to the same file 
as I mentioned in the CASSANDRA-8476 so if there is no way to tell how 
reasonable is it to enforce shutdown or any rule from disk_failure_policy?

bq. If it's our bug, then you may need a temporary patch while we figure out 
the cause, but I still don't think that kind of // this shouldn't happen code 
should be shipped officially.

If it's your problem it's my problem as well, we can work around for now (as I 
guess most of the people do) but my intention in this ticket to fix this 
problem for good instead of just fixing the symptom of it (being aforementioned 
duplicate hard-link problem).

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 2.0.12

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-06-19 Thread Yuki Morishita (JIRA)

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

Yuki Morishita edited comment on CASSANDRA-7275 at 6/19/14 4:57 PM:


Patch removes latch completely and use ListenableFuture instead. This way we 
can submit post flush process only if related flushes succeed.
(patch is against 2.0 branch, but maybe 2.1 is suitable for the change)


was (Author: yukim):
Patch removes latch completely and use ListenableFuture instead. This way we 
can submit post flush process only if related flushes succeed.

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Mikhail Stepura
Priority: Minor
 Fix For: 1.2.17, 2.0.9

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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


[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

2014-06-19 Thread Yuki Morishita (JIRA)

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

Yuki Morishita edited comment on CASSANDRA-7275 at 6/19/14 4:56 PM:


Patch removes latch completely and use ListenableFuture instead. This way we 
can submit post flush process only if related flushes succeed.


was (Author: yukim):
Path removes latch completely and use ListenableFuture instead. This way we can 
submit post flush process only if related flushes succeed.

 Errors in FlushRunnable may leave threads hung
 --

 Key: CASSANDRA-7275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Mikhail Stepura
Priority: Minor
 Fix For: 1.2.17, 2.0.9

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 7252-2.0-v2.txt


 In Memtable.FlushRunnable, the CountDownLatch will never be counted down if 
 there are errors, which results in hanging any threads that are waiting for 
 the flush to complete.  For example, an error like this causes the problem:
 {noformat}
 ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 
 198) Exception in thread Thread[FlushWriter:474,5,main]
 java.lang.IllegalArgumentException
 at java.nio.Buffer.position(Unknown Source)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
 at 
 org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
 at 
 org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
 at 
 org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
 at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
 at 
 org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
 at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
 {noformat}



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