----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/20698/#review41576 -----------------------------------------------------------
flume-ng-doc/sphinx/FlumeUserGuide.rst <https://reviews.apache.org/r/20698/#comment75097> AFAICT, "Unlimited" is not actually a valid value to have in a config file, so I think people may get confused when trying to specify it. How about using 0 as the default? Of course, we'd need to indicate in the docs that 0 means it will retry forever. I'd also humbly suggest a rename of this property to "maxCloseAttempts" if we do the above, since it should be intuitive that 0 would not make sense for max # of close attempts. I do agree that conceptually, unlimited is the right default. Minor nit: end-of-line whitespace on this line flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75101> I don't think a single static instance is going to work with different FileSystem impls, say in a federated case, or in a situation where we have some sinks writing to HDFS and some sinks writing to S3 or a local FS. Style note: All this detection / reflection stuff should be factored into a subroutine (this method is already long enough). Because we are not able to use a JVM static for the Method, let's also avoid doing the reflection under the static lock. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75102> Please factor creation of the callable into a subroutine as well, so we don't make close() greater than a screenful of code. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75104> Hmm, how about "== false"? Also, this would read more easily if isClosedMethod.invoke(...) was factored into a private method that could be called more naturally... directly invoking on the reflection stuff is additionally awkward. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75103> The logic is a little wonky here... if there is no close failure but there is a rename failure, this will potentially retry forever. Either the failure scenarios need to be separated out, or this should respect the max close tries config param. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75107> Do we need to null this out? There should not be a circular ref from FS to this Callable, right? flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75113> s/minutes/seconds/ flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75116> Why are we nulling-out fileSystem all over the place? I may actually be guilty of doing this sillyness in the past.. but we should get away from this. I think we should consider this instance of FileSystem to be owned by BucketWriter, and be aggressive about getting rid of refs to BucketWriter from stuff like the HDFSSink when they are no longer needed. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75118> Worth a comment that the BucketWriter may be kept alive, and that this method may be called by the scheduledClose Callable on the timedRollerPool even after the BucketWriter is closed. However, since there is an implicit ref to the BucketWriter from the Callable, I don't see the point of copying in all these args when we could get them from the BucketWriter object. flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java <https://reviews.apache.org/r/20698/#comment75115> Nit: This could be <Void> typed and be forced to return null flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java <https://reviews.apache.org/r/20698/#comment75119> These tests have a lot of copy/paste, can this be reduced? - Mike Percy On April 25, 2014, 5:11 a.m., Hari Shreedharan wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/20698/ > ----------------------------------------------------------- > > (Updated April 25, 2014, 5:11 a.m.) > > > Review request for Flume. > > > Bugs: FLUME-2357 > https://issues.apache.org/jira/browse/FLUME-2357 > > > Repository: flume-git > > > Description > ------- > > Much of the size of the patch is due to a couple of file renames. Otherwise > the patch itself is pretty simple. In the Bucketwriter, if a close fails, we > simply reschedule the close to happen sometime later until it finally > succeeds or till we hit a maximum count. Added a test case too. This depends > on the presence of the isFileClosed method in the HDFS client API. If the > method is absent, reattempts are not done. > > > Diffs > ----- > > flume-ng-doc/sphinx/FlumeUserGuide.rst 7b918ed > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/AbstractHDFSWriter.java > da0466d > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java > e82d13d > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSCompressedDataStream.java > 5518547 > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSDataStream.java > e20d1ee > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java > 4ea78c1 > > flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSSequenceFile.java > 5fe9f1b > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockDataStream.java > PRE-CREATION > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystem.java > PRE-CREATION > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFileSystemCloseRetryWrapper.java > b5d89e6 > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStream.java > PRE-CREATION > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/MockFsDataOutputStreamCloseRetryWrapper.java > 1d8c140 > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java > b7cc586 > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java > 87918d1 > > flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestUseRawLocalFileSystem.java > 4476530 > pom.xml 2aa0ad1 > > Diff: https://reviews.apache.org/r/20698/diff/ > > > Testing > ------- > > Added new unit test. All current tests pass. > > > Thanks, > > Hari Shreedharan > >
