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

          Inder SIngh created issue -
          Inder SIngh made changes -
          Field Original Value New Value
          Link This issue is a clone of FLUME-1080 [ FLUME-1080 ]
          Hide
          Inder SIngh added a comment -

          Folks,

          was browsing through and found that HDFSCompressedDataStream was added.
          open() would hit the same issue as reported in https://issues.apache.org/jira/browse/FLUME-1080
          i.e. the corner case of trying to do an append in a non existent file.

          Show
          Inder SIngh added a comment - Folks, was browsing through and found that HDFSCompressedDataStream was added. open() would hit the same issue as reported in https://issues.apache.org/jira/browse/FLUME-1080 i.e. the corner case of trying to do an append in a non existent file.
          Hide
          jiraposter@reviews.apache.org added a comment -

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

          Review request for Flume and Arvind Prabhakar.

          Summary
          -------

          HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file.

          This addresses bug FLUME-1112.
          https://issues.apache.org/jira/browse/FLUME-1112

          Diffs


          trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206

          Diff: https://reviews.apache.org/r/4683/diff

          Testing
          -------

          Ran through the test cases of FLUME.

          Thanks,

          Inder

          Show
          jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4683/ ----------------------------------------------------------- Review request for Flume and Arvind Prabhakar. Summary ------- HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file. This addresses bug FLUME-1112 . https://issues.apache.org/jira/browse/FLUME-1112 Diffs trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206 Diff: https://reviews.apache.org/r/4683/diff Testing ------- Ran through the test cases of FLUME. Thanks, Inder
          Hide
          Inder SIngh added a comment -

          Patch to address the case when append=true and it tries to write to a non existent file.

          Show
          Inder SIngh added a comment - Patch to address the case when append=true and it tries to write to a non existent file.
          Inder SIngh made changes -
          Attachment FLUME-1112-1.patch [ 12521945 ]
          Hide
          Inder SIngh added a comment -

          Please review the patch at - https://reviews.apache.org/r/4683/
          Also attached along with this JIRA.

          Testing Done

          1. Unit tests of FLUME

          Show
          Inder SIngh added a comment - Please review the patch at - https://reviews.apache.org/r/4683/ Also attached along with this JIRA. Testing Done 1. Unit tests of FLUME
          Inder SIngh made changes -
          Status Open [ 1 ] Patch Available [ 10002 ]
          Hide
          jiraposter@reviews.apache.org added a comment -

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

          +1 looks good to me

          • Mike

          On 2012-04-09 12:24:59, Inder Singh wrote:

          -----------------------------------------------------------

          This is an automatically generated e-mail. To reply, visit:

          https://reviews.apache.org/r/4683/

          -----------------------------------------------------------

          (Updated 2012-04-09 12:24:59)

          Review request for Flume and Arvind Prabhakar.

          Summary

          -------

          HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file.

          This addresses bug FLUME-1112.

          https://issues.apache.org/jira/browse/FLUME-1112

          Diffs

          -----

          trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206

          Diff: https://reviews.apache.org/r/4683/diff

          Testing

          -------

          Ran through the test cases of FLUME.

          Thanks,

          Inder

          Show
          jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4683/#review6933 ----------------------------------------------------------- +1 looks good to me Mike On 2012-04-09 12:24:59, Inder Singh wrote: ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4683/ ----------------------------------------------------------- (Updated 2012-04-09 12:24:59) Review request for Flume and Arvind Prabhakar. Summary ------- HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file. This addresses bug FLUME-1112 . https://issues.apache.org/jira/browse/FLUME-1112 Diffs ----- trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206 Diff: https://reviews.apache.org/r/4683/diff Testing ------- Ran through the test cases of FLUME. Thanks, Inder
          Hide
          jiraposter@reviews.apache.org added a comment -

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

          Ship it!

          lgtm

          • Brock

          On 2012-04-09 12:24:59, Inder Singh wrote:

          -----------------------------------------------------------

          This is an automatically generated e-mail. To reply, visit:

          https://reviews.apache.org/r/4683/

          -----------------------------------------------------------

          (Updated 2012-04-09 12:24:59)

          Review request for Flume and Arvind Prabhakar.

          Summary

          -------

          HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file.

          This addresses bug FLUME-1112.

          https://issues.apache.org/jira/browse/FLUME-1112

          Diffs

          -----

          trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206

          Diff: https://reviews.apache.org/r/4683/diff

          Testing

          -------

          Ran through the test cases of FLUME.

          Thanks,

          Inder

          Show
          jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4683/#review7220 ----------------------------------------------------------- Ship it! lgtm Brock On 2012-04-09 12:24:59, Inder Singh wrote: ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4683/ ----------------------------------------------------------- (Updated 2012-04-09 12:24:59) Review request for Flume and Arvind Prabhakar. Summary ------- HDFSCompressedDataStream append will fail when append=true and it tries to append to a nonexistent file. This addresses bug FLUME-1112 . https://issues.apache.org/jira/browse/FLUME-1112 Diffs ----- trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java 1311206 Diff: https://reviews.apache.org/r/4683/diff Testing ------- Ran through the test cases of FLUME. Thanks, Inder
          Brock Noland made changes -
          Summary CLONE - Issue with HDFSEventSink for append support for HDFSCompressedDataStream HDFSCompressedDataStream append does not work
          Hide
          Brock Noland added a comment -

          Committed in 1330412.

          Show
          Brock Noland added a comment - Committed in 1330412.
          Brock Noland made changes -
          Status Patch Available [ 10002 ] Resolved [ 5 ]
          Resolution Fixed [ 1 ]
          Hide
          Hudson added a comment -

          Integrated in flume-trunk #186 (See https://builds.apache.org/job/flume-trunk/186/)
          FLUME-1112: HDFSCompressedDataStream append does not work

          (Inder Singh via Brock Noland) (Revision 1330412)

          Result = SUCCESS
          brock : http://svn.apache.org/viewvc/?view=rev&rev=1330412
          Files :

          • /incubator/flume/trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java
          Show
          Hudson added a comment - Integrated in flume-trunk #186 (See https://builds.apache.org/job/flume-trunk/186/ ) FLUME-1112 : HDFSCompressedDataStream append does not work (Inder Singh via Brock Noland) (Revision 1330412) Result = SUCCESS brock : http://svn.apache.org/viewvc/?view=rev&rev=1330412 Files : /incubator/flume/trunk/flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java

            People

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

              Dates

              • Created:
                Updated:
                Resolved:

                Development