Flume
  1. Flume
  2. FLUME-1112

HDFSCompressedDataStream append does not work

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: v1.2.0
    • Component/s: None
    • Labels:
      None

      Description

      Enabled append feature in HDFS and flume conf

      Hit the following exceptions -

      2012-03-29 13:27:03,284 (SinkRunner-PollingRunner-DefaultSinkProcessor) [WARN - org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:370)] HDFS IO error
      java.io.FileNotFoundException: java.io.FileNotFoundException: failed to append to non-existent file /flume.0.tmp on client 127.0.0.1
      org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:586)
      at org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:209)
      at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:630)
      at org.apache.flume.sink.hdfs.HDFSDataStream.open(HDFSDataStream.java:42)
      at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:82)
      at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:102)
      at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:342)
      at org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:65)
      at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:148)
      at java.lang.Thread.run(Thread.java:680)
      Caused by: org.apache.hadoop.ipc.RemoteException: java.io.FileNotFoundException: failed to append to non-existent file /flume.0.tmp on client 127.0.0.1
      at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:1187)
      at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.appendFile(FSNamesystem.java:1357)
      at org.apache.hadoop.hdfs.server.namenode.NameNode.append(NameNode.java:600)
      at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)

      Looking at the code and debugging further looks like the case of first write when append="true" isn't handled.
      Here is the code in picture -> HDFSDataStream.open

      if (conf.getBoolean("hdfs.append.support", false) == true)

      { outStream = hdfs.append(dstPath); } else { outStream = hdfs.create(dstPath); }

      it should be something like this -
      if (conf.getBoolean("hdfs.append.support", false) == true && hdfs.isFile(dstPath)) { outStream = hdfs.append(dstPath); }

      else

      { outStream = hdfs.create(dstPath); }

      Refer - http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/fs/FileSystem.html - append works on an existent file.

      Pardon my ignorance and correct me if am missing something here

        Issue Links

          Activity

          No work has yet been logged on this issue.

            People

            • Assignee:
              Inder SIngh
              Reporter:
              Inder SIngh
            • Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development