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

Amareshwari Sriramadasu commented on HDFS-3533:
-----------------------------------------------

Thanks Todd for the inputs. Will look into DN logs and update the issue.

Elaborating more on the usecase : The file is getting flushed every second on 
the producer. There is consumer tailing the file every 5 seconds. Below is the 
code snippet of the consumer:
{noformat}

{ 
  FSDataInputStream inStream;
  BufferedReader reader;
  long currentOffset = 0;

  while (true) {
    if (reader != null) reader.close();
    if (inStream != null) inStream.close();

    inStream = fs.open(mypath);
    reader = new BufferedReader(new InputStreamReader(in));
    inStream.seek(currentOffset);


    while (true) {
      line = reader.readLine();

      if (line == null) {
        Thread.sleep(5000);
        break;
      } else {
        currentOffset = inStream.getPos();
        .....
      }
    }
  }
}
{noformat}

We are hitting NPE in our production cluster atmost once every one hour, where 
the file produced is of size 100-150MB per minute. File could be opened 
properly after retry.

We are also hitting the following Exception more often than NPE (5 times in one 
hour):
{noformat}
java.io.IOException: Cannot seek after EOF
        at
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.seek(DFSClient.java:2149)
        at
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:37)
{noformat}

Seeking also works properly after retry. 

The second exception looks like another race. If they are not related, I can 
raise another issue for the second exception.

                
> NPE in DFSClient$DFSInputStream.openInfo
> ----------------------------------------
>
>                 Key: HDFS-3533
>                 URL: https://issues.apache.org/jira/browse/HDFS-3533
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs client
>    Affects Versions: 0.20.203.0
>            Reporter: Amareshwari Sriramadasu
>
> I'm seeing the following NPE:
> java.lang.NullPointerException
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSInputStream.openInfo(DFSClient.java:1633)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSInputStream.<init>(DFSClient.java:1593)
>         at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:428)
>         at
> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:187)
>         at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:456)
> Here, the file gets opened while it's being written.   

--
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

        

Reply via email to