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

stack commented on HBASE-20507:
-------------------------------

I pushed addendum (but did not call it out as so in the commit message....). 
[~Apache9] Let me know if I have this wrong.

{code}
commit 09ca6193890e8caffeb2951340b6a6fc10b5cfa0 (HEAD -> m, origin/master, 
origin/HEAD)
Author: Michael Stack <[email protected]>
Date:   Thu May 3 12:26:12 2018 -0700

    HBASE-20507 Do not need to call recoverLease on the broken file when we 
fail to create a wal writer

diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestOverwriteFileUnderConstruction.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestOverwriteFileUnderConstruction.java
index 7f75fdb268..aaeed1523f 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestOverwriteFileUnderConstruction.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestOverwriteFileUnderConstruction.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;

+import java.io.FileNotFoundException;
 import java.io.IOException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -95,6 +96,8 @@ public class TestOverwriteFileUnderConstruction {
       out1.close();
       // a successful close is also OK for us so no assertion here, we just 
need to confirm that the
       // data in the file are correct.
+    } catch (FileNotFoundException fnfe) {
+      // hadoop3 throws one of these.
     } catch (RemoteException e) {
       // expected
       assertThat(e.unwrapRemoteException(), 
instanceOf(LeaseExpiredException.class));
{code}

> Do not need to call recoverLease on the broken file when we fail to create a 
> wal writer
> ---------------------------------------------------------------------------------------
>
>                 Key: HBASE-20507
>                 URL: https://issues.apache.org/jira/browse/HBASE-20507
>             Project: HBase
>          Issue Type: Improvement
>          Components: wal
>            Reporter: Duo Zhang
>            Assignee: Duo Zhang
>            Priority: Major
>             Fix For: 2.1.0, 2.0.1
>
>         Attachments: 20507.addendum.patch, HBASE-20507.patch
>
>
> I tried locally with a UT, if we overwrite a file which is currently being 
> written, the old file will be completed and then deleted. If you call close 
> on the previous file, a no lease exception will be thrown which means that 
> the file has already been completed.
> So we do not need to close a file if it will be overwritten immediately, 
> since recoverLease may take a very long time...



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to