Author: asrabkin
Date: Tue Mar  3 23:19:15 2009
New Revision: 749833

URL: http://svn.apache.org/viewvc?rev=749833&view=rev
Log:
HADOOP-5370
SeqFileWriter won't write empty sink files.  (patch by asrabkin)

Modified:
    
hadoop/core/trunk/src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java

Modified: 
hadoop/core/trunk/src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
URL: 
http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java?rev=749833&r1=749832&r2=749833&view=diff
==============================================================================
--- 
hadoop/core/trunk/src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
 (original)
+++ 
hadoop/core/trunk/src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
 Tue Mar  3 23:19:15 2009
@@ -75,6 +75,7 @@
        
        private int initWriteChunkRetries = 10;
        private int writeChunkRetries = initWriteChunkRetries;
+       private boolean chunksWrittenThisRotate = false;
        
        public SeqFileWriter() throws WriterException
        {
@@ -227,18 +228,24 @@
                                if (previousOutputStr != null)  
                                {
                                        previousOutputStr.close();
-                                       fs.rename(previousPath,
-                                                       new 
Path(previousFileName + ".done"));
+                                       if(chunksWrittenThisRotate) {
+                                         fs.rename(previousPath, new 
Path(previousFileName + ".done"));
+                                       } else {
+                                         log.info("no chunks written to "+ 
previousPath + ", deleting");
+                                         fs.delete(previousPath, false);
+                                       }
                                }
                                Path newOutputPath = new Path(newName + 
".chukwa");                     
                                FSDataOutputStream newOutputStr = 
fs.create(newOutputPath);
                                currentOutputStr = newOutputStr;
                                currentPath = newOutputPath;
                                currentFileName = newName;
+                               chunksWrittenThisRotate = false;
                                // Uncompressed for now
                                seqFileWriter = SequenceFile.createWriter(conf, 
newOutputStr,
                                                ChukwaArchiveKey.class, 
ChunkImpl.class,
                                                
SequenceFile.CompressionType.NONE, null);
+                               
                        }
                        catch (IOException e)
                        {
@@ -261,10 +268,9 @@
        @Override
        public void add(List<Chunk> chunks) throws WriterException
        {
-               if (chunks != null)     
-               {
-                       try 
-                       {
+               if (chunks != null) {
+                       try {
+                   chunksWrittenThisRotate = true;
                                ChukwaArchiveKey archiveKey = new 
ChukwaArchiveKey();
 
                                // FIXME compute this once an hour


Reply via email to