[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-27 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

Okay.  Made some minor updates to v6 and committed.

Still to fix:
- SequentialWriter has a lot of IOExceptions that should be converted to FSWE


 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 2116-v5.txt, 
 CASSANDRA-2116-v3.patch, CASSANDRA-2116-v4.patch, CASSANDRA-2116-v6.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-26 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-2116:
--

The patch is getting large. If you find that something is still missing, but 
that what is present is good enough, then it probably makes sense to merge this 
one and add the missing fixes in a separate patch.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 2116-v5.txt, 
 CASSANDRA-2116-v3.patch, CASSANDRA-2116-v4.patch, CASSANDRA-2116-v6.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-2116:


bq. Looks like we should drop the throws IOException declaration from 
applyIndexUpdates (and have that chain throw FSWE as needed).

If we do so, I think we need to get message about throwing FSWE to the javadoc 
for the method to get API users attention to change their code appropriately.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-2116:
--

{quote}
Looks like we should drop the throws IOException declaration from 
applyIndexUpdates (and have that chain throw FSWE as needed).
{quote}
I think *SecondaryIndex methods should throw some kind of 
SecondaryIndexException instead.

{quote}
SSTII should throw IOException when it doesn't know what DataInput is. Callers 
can transform to FSRE. (Other constructors, or in the last case, 
IncomingStreamReader.)
{quote}
Do you mean the constructor only? Because other methods (next, reset) in SSTII 
implement Iterator/ICountableColumnIterator. Can't throw IOE. Can modify 
ICountableColumnIterator, but not Iterator, for obvious reasons. Maybe throw 
IOException from the constructor, but use instanceof and throw FSReadError in 
reset and next?

{quote}
BTW: congratulations on getting import ordering (almost) correct on the first 
try. The only thing missing is, com.google.common goes above org.slf4j instead 
of being lumped in with everything else.
{quote}
Thanks. Had to do _something_ almost right.


 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

bq. I think *SecondaryIndex methods should throw some kind of 
SecondaryIndexException instead

Why would that be more useful?

bq. Do you mean the constructor only?

Yes.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-2116:
--

{quote}
Why would that be more useful?
{quote}
In some implementations (not talking about KeysIndex) it's not just IO that 
might go wrong. I don't want to make people choose between wrapping the real 
exception in IOE or RTE or swallowing everything.
There is at least one other implementation - in DSE, probably other ones, so 
it's not a completely made up issue.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

bq. I don't want to make people choose between wrapping the real exception in 
IOE or RTE or swallowing everything.

The status quo is IOE or RTE.  IOE is superseded by FSWE, and RTE can stay 
the same.  If we need to improve things beyond that, it's out of scope for this 
ticket.

bq. Actually it can't throw 

You're right.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-21 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-2116:
--

{quote}
The status quo is IOE or RTE. IOE is superseded by FSWE, and RTE can stay the 
same. If we need to improve things beyond that, it's out of scope for this 
ticket.
{quote}
Just to be clear: by IOE here I meant IOException, not IOError. I was thinking 
about KeysIndex#forceBlockingFlush that is (was) wrapping ExecutionException 
AND InterruptedException in IOException just because the interface allowed only 
IOEx.
But I agree with you regarding drop the throws IOException declaration from 
applyIndexUpdates. At one point I actually did just that. Will do.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-20 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

DD.createAllDirectories will stop trying to create as soon as the first 
directory fails, so it's not going to be appropriate for generic FSWriteError 
handling.  Suggest logging an error and explicitly shutting down instead.  
(This should only be called on startup.)

Looks like we should drop the throws IOException declaration from 
applyIndexUpdates (and have that chain throw FSWE as needed).

BatchCommitLogExecutorService.processWithSyncBatch should throw FSWE instead of 
RTE.  

CommitLogSegment.sync should turn IOE into FSWE.  Rest of sync heirarchy won't 
need throws declaration.

Note for CASSANDRA-2118: will need to unwrap exceptions looking for FSWE since 
CLES/PCLES can wrap in ExecutionException.  (Others might as well.  Easier to 
do unwrap check in 2118 than to audit all possible executors.)  On the other 
hand, this makes trying to catch the error before it hits the exception hook 
more of a pain, as in the next item...

CollationController needs to retain its try/catch, since we want to allow the 
read to succeed, even if the defragmenting write fails.  Since it could error 
w/ either FSWE or EE (from the commitlog add), probably need to catch generic 
Exception.  For 2118 we can add some way to submit this to the disk blacklister 
without re-throwing.

Looks like it would be worth adding a constructor for FSRW taking a Descriptor.

SSTR.createLinks should throw FSWE.

Methods called by SSTW constructor should throw FSWE.

SSTW methods should throw FSWE. (callers of append will want to catch + 
re-throw after cleanup.)

TruncateVerbHandler (and anyone else) shouldn't swallow potential FSWE by 
logging, need to rethrow.  (FBUtilities.unchecked is handy in such cases.)

I agree with your use of AssertionError in LCR.  Would prefer to use RTE in 
SSTableReader though, since we do some tricky reference counting around that 
and I wouldn't want to ignore problems there b/c someone turned off assertions. 
 (Surprisingly common...)

SSTII should throw IOException when it doesn't know what DataInput is.  Callers 
can transform to FSRE.  (Other constructors, or in the last case, 
IncomingStreamReader.)

Corrupt sstables (sstablescanner + others?) shouldn't be turned into FSRE, 
since it's usually bad memory or a bug and not the disk's fault.

FileUtils should throw FSWE.

BTW: congratulations on getting import ordering (almost) correct on the first 
try.  The only thing missing is, com.google.common goes above org.slf4j instead 
of being lumped in with everything else.


 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch, 
 CASSANDRA-2116-v3.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-16 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

Iterators are screwing us.  Ideally I'd like to have things like BSF.getSegment 
just throw IOException back up until someone who knows what sstable it belongs 
to, can deal with it, but since Iterator.next can't throw anything [checked] 
that's not an option.

So I think what we need to do is go back to FSReadError / FSWriteError, with 
just a path to the file in question, and let the exception handler deal with 
parsing out directory + ks + cf from that.

This has the added benefit of being able to handle non-sstable i/o (cache 
saving, commitlogs) without needing extra exception types.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-16 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-2116:


+1 with Jonathan, that would also allow as us throw less RTE exceptions like, 
for example, in SystemTable or ReadRepairVerbHandler (associated with 
RowMutation.apply() calls). I also think that we should keep length() method 
from RandomAccessReader throwing IOException, as RAF does originally, because I 
don't think we want to keep future extenders from using custom (potentially I/O 
based) length tracking. Nit: license header is missing from all of the new 
files.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-07-16 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

bq. SerializationError seems to be a good idea for the situations where action 
is not actually FS related

I'm fine leaving these as RTE since we can't do anything reasonable to recover 
from them either way.

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Chris Goffinet
Assignee: Aleksey Yeschenko
 Fix For: 1.2

 Attachments: 
 0001-Issue-2116-Replace-some-IOErrors-with-more-informati.patch, 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2012-05-28 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

I think you're probably right.  Ideally what I'd like is something like this:

- Code that knows what disk is involved throws an FSReadError/FSWriteError that 
*records the disk/volume in question*
- Notably this will not necessarily be the lowest-level code, which often just 
takes a DataInput or DataOutput interface.  We'll want to declare the checked 
IOException at that level to make sure we handle it higher up
- Our global uncaught exception handler can mark the disks in question bad 
(exactly where CASSANDRA-2118 looks to terminate things; could handle this over 
there to go with the existing division of code)
- As for the request that hit the exception in the first place, we just allow 
that to fail + timeout normally; little benefit to be had by complicating that 
further (which is the approach taken here and 2118; I just note it to be 
explicit)

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
Reporter: Chris Goffinet
Priority: Minor
 Fix For: 1.2

 Attachments: 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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




[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2011-05-17 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2116:
---

I'm not sure having different classes for read/write errors is necessary (code 
that is in a position to catch-and-do-something-reasonable knows what kind of 
op it's attempting). On the other hand, if a write op does a read as part of 
its implementation (indexes cause this to happen) we might need to distinguish 
the two.

I think it's more useful to distinguish between recoverable errors and non-: I 
got EOF earlier than I thought usually means the file is corrupt not the disk 
is dead.  (I can't think of any read errors that absolutely mean disk-is-dead.)

It would be useful to get some use out of Java's misguided checked exceptions, 
by keeping recoverable errors checked (IOException) and unrecoverable ones 
unchecked (IOError).

 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
Reporter: Chris Goffinet
Priority: Minor
 Fix For: 1.0

 Attachments: 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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


[jira] [Commented] (CASSANDRA-2116) Separate out filesystem errors from generic IOErrors

2011-05-17 Thread Chris Goffinet (JIRA)

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

Chris Goffinet commented on CASSANDRA-2116:
---

Unfortunately the best we can get is IOError from Java. For example we use this 
patch to actually detect when our raid array dies, the OS will tell java to 
throw IOError. I think we should error on the side of, if data is corrupt, we 
should let the operator decide what mode he wants. For us, any errors or any 
corruption of data, we want to take the node out right away.

We have been testing this in production for awhile and it works really well 
when disks die, and we also did tests involving removing drives from the system 
while it was serving traffic. 

The Read/Write classes was a similar idea of how the Hadoop code base handles 
this very issue.


 Separate out filesystem errors from generic IOErrors
 

 Key: CASSANDRA-2116
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2116
 Project: Cassandra
  Issue Type: Improvement
Reporter: Chris Goffinet
Priority: Minor
 Fix For: 1.0

 Attachments: 
 0001-Separate-out-filesystem-errors-from-generic-IOErrors.patch


 We throw IOErrors everywhere today in the codebase. We should separate out 
 specific errors such as (reading, writing) from filesystem into FSReadError 
 and FSWriteError. This makes it possible in the next ticket to allow certain 
 failure modes (kill the server if reads or writes fail to disk).

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