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

[email protected] commented on HBASE-4014:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/969/#review1667
-----------------------------------------------------------


I think there's still one important tweak to the exception handling -- if we 
trigger a server abort on a coprocessor exception, we should make sure that an 
exception is thrown back to the client.  Otherwise there's no indication on the 
client side that any problem occurred.

Other than that, there are some style problems throughout the patch to clean up:

- brace alignment: should be "} else {" and "} catch (...) {" -- on same line 
as closing brace
- spaces follow commas in method parameters: "method(arg1, arg2)"
- limit lines to 80 chars
- some unnecessary empty lines


src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3747>

    Combine into a single comment?



src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3743>

    Move up to prev line:  } else {



src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3759>

    I think the name of the config variable should be 
hbase.coprocessor.abortonerror.  No other hbase configuration property uses 
underscores in the name.
    
    Also, you can just use:
    if (conf.getBoolean("hbase.coprocessor.abortonerror", false)) {
    
    since CoprocessorHost has it's own instance-level Configuration reference.



src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3760>

    Move up to prev line



src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3837>

    This should be below the else block so it's always thrown.
    
    In the case the region server is aborted, we actually want an exception to 
come back to the client.  Otherwise we're burying the error and the client just 
assumes the operation succeeded, which is bad.



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3762>

    Should be on prev line



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3766>

    Args should have a space after a comma



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3761>

    Should be on prev line



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3763>

    Empty line



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3764>

    Empty line



src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3765>

    Prev line



src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
<https://reviews.apache.org/r/969/#comment3767>

    Limit lines to 80 char
    
    hookName isn't used here, do we need it?  Stack trace will point to the 
method name anyway.



src/main/resources/hbase-default.xml
<https://reviews.apache.org/r/969/#comment3768>

    As stated above, I think it should be hbase.coprocessor.abortonerror



src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorException.java
<https://reviews.apache.org/r/969/#comment3769>

    Should this check the type of exception that's expected?



src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorException.java
<https://reviews.apache.org/r/969/#comment3835>

    Does this really need to be a thread?  Why not just do the put 
synchronously?



src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorException.java
<https://reviews.apache.org/r/969/#comment3836>

    If the put triggers a RS abort, I think we should be expecting an exception 
here.  Wouldn't we want to make sure it's thrown to notify the client of the 
error?


- Gary


On 2011-08-26 20:52:06, Eugene Koontz wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/969/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-26 20:52:06)
bq.  
bq.  
bq.  Review request for hbase, Gary Helmling and Mingjie Lai.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  https://issues.apache.org/jira/browse/HBASE-4014 Coprocessors: Flag the 
presence of coprocessors in logged exceptions
bq.  
bq.  The general gist here is to wrap each of 
{Master,RegionServer}CoprocessorHost's coprocessor call inside a 
bq.  
bq.  "try { ... } catch (Throwable e) { handleCoprocessorThrowable(e) }"
bq.  
bq.  block. 
bq.  
bq.  handleCoprocessorThrowable() is responsible for either passing 'e' along 
to the client (if 'e' is an IOException) or, otherwise, aborting the service 
(Regionserver or Master).
bq.  
bq.  The abort message contains a list of the loaded coprocessors for crash 
analysis.
bq.  
bq.  
bq.  This addresses bug HBASE-4014.
bq.      https://issues.apache.org/jira/browse/HBASE-4014
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java 
7a3ac1d 
bq.    src/main/java/org/apache/hadoop/hbase/master/HMaster.java 92d5dbb 
bq.    src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java 
aa930f5 
bq.    src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 
58f0350 
bq.    
src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java 
53645ce 
bq.    src/main/resources/hbase-default.xml 2c8f44b 
bq.    
src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorException.java
 PRE-CREATION 
bq.    
src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorException.java
 PRE-CREATION 
bq.  
bq.  Diff: https://reviews.apache.org/r/969/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  patch includes two tests:
bq.  
bq.  TestMasterCoprocessorException.java
bq.  TestRegionServerCoprocessorException.java
bq.  
bq.  both tests pass in my build environment.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Eugene
bq.  
bq.



> Coprocessors: Flag the presence of coprocessors in logged exceptions
> --------------------------------------------------------------------
>
>                 Key: HBASE-4014
>                 URL: https://issues.apache.org/jira/browse/HBASE-4014
>             Project: HBase
>          Issue Type: Improvement
>          Components: coprocessors
>            Reporter: Andrew Purtell
>            Assignee: Eugene Koontz
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4014.patch, HBASE-4014.patch, HBASE-4014.patch, 
> HBASE-4014.patch, HBASE-4014.patch
>
>
> For some initial triage of bug reports for core versus for deployments with 
> loaded coprocessors, we need something like the Linux kernel's taint flag, 
> and list of linked in modules that show up in the output of every OOPS, to 
> appear above or below exceptions that appear in the logs.

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

        

Reply via email to