[
https://issues.apache.org/jira/browse/HDFS-7342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14520401#comment-14520401
]
Ravi Prakash commented on HDFS-7342:
------------------------------------
I found another\(?) instance in which the lease is not recovered. This is
reproducible easily on a pseudo-distributed single node cluster
# Before you start it helps if you set. This is not necessary, but simply
reduces how long you have to wait
{code}
public static final long LEASE_SOFTLIMIT_PERIOD = 30 * 1000;
public static final long LEASE_HARDLIMIT_PERIOD = 2 * LEASE_SOFTLIMIT_PERIOD;
{code}
# Client starts to write a file. (could be less than 1 block, but it hflushed
so some of the data has landed on the datanodes) (I'm copying the client code I
am using. I generate a jar and run it using $ hadoop jar TestHadoop.jar)
# Client crashes. (I simulate this by kill -9 the $(hadoop jar TestHadoop.jar)
process after it has printed "Wrote to the bufferedWriter"
# Shoot the datanode. (Since I ran on a pseudo-distributed cluster, there was
only 1)
I believe the lease should be recovered and the block should be marked missing.
However this is not happening. The lease is never recovered. I am going to
check what happens when only the primary datanode is shot. {color:red}Please
let me know if I shouldn't hijack this JIRA. By default I will{color}
{code:title=TestHadoop.java|borderStyle=solid}
import java.io.BufferedWriter;
import java.io.IOException;
import java.io.OutputStreamWriter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
public class TestHadoop {
public static void main(String args[]) throws IOException,
InterruptedException {
Path path = new Path("/tmp/testHadoop");
Configuration conf = new Configuration();
FileSystem fs = FileSystem.get(conf);
System.out.println("DefaultFS: " + conf.get("fs.defaultFS"));
System.out.flush();
FSDataOutputStream hdfsout = fs.create(path,true);
BufferedWriter br=new BufferedWriter(new OutputStreamWriter(hdfsout));
System.out.println("Created the bufferedWriter" );
System.out.flush();
br.write("Some string");
br.flush();
hdfsout.hflush();
System.out.println("Wrote to the bufferedWriter" );
System.out.flush();
Thread.sleep(120000); //KILL THE PROCESS DURING THIS SLEEP
br.close();
System.out.println("Closed the bufferedWriter" );
System.out.flush();
}
}
{code}
> Lease Recovery doesn't happen some times
> ----------------------------------------
>
> Key: HDFS-7342
> URL: https://issues.apache.org/jira/browse/HDFS-7342
> Project: Hadoop HDFS
> Issue Type: Bug
> Affects Versions: 2.0.0-alpha
> Reporter: Ravi Prakash
> Attachments: HDFS-7342.1.patch, HDFS-7342.2.patch, HDFS-7342.3.patch
>
>
> In some cases, LeaseManager tries to recover a lease, but is not able to.
> HDFS-4882 describes a possibility of that. We should fix this
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)