[
https://issues.apache.org/jira/browse/CHUKWA-4?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12853188#action_12853188
]
Ahmed Fathalla commented on CHUKWA-4:
-------------------------------------
I am working on a helper class that could be called before the moveFile method
in LocalToRemoteHDFSMover, the purpose of this helper would be to copy the
contents of the (possibly corrupt) .chukwa file into a valid sink file. Here is
the class I created
public class CopySequenceFile {
static Logger log = Logger.getLogger(LocalWriter.class);
private static SequenceFile.Writer seqFileWriter = null;
private static SequenceFile.Reader seqFileReader = null;
private static FSDataOutputStream newOutputStr = null;
public static void main(String args[]){
}
public static void createValidSequenceFile(Configuration conf, String
originalFileDir, String originalFileName,FileSystem localFs){
try{
String originalCompleteDir= originalFileDir + originalFileName;
Path originalPath= new Path (originalCompleteDir);
int extensionIndex= originalFileName.indexOf(".chukwa",0);
String newFileName=originalFileName.substring(0,
extensionIndex)+".recover";
String newCompleteDir= originalFileDir+ newFileName;
Path newPath= new Path (newCompleteDir);
newOutputStr = localFs.create(newPath);
seqFileWriter = SequenceFile.createWriter(conf, newOutputStr,
ChukwaArchiveKey.class, ChunkImpl.class,
SequenceFile.CompressionType.NONE, null);
seqFileReader = new SequenceFile.Reader (localFs, originalPath, conf);
System.out.println("key class name is " +
seqFileReader.getKeyClassName());
System.out.println("value class name is " +
seqFileReader.getValueClassName());
ChukwaArchiveKey key = new ChukwaArchiveKey();
ChunkImpl evt = ChunkImpl.getBlankChunk();
while (seqFileReader.next(key, evt)){
seqFileWriter.append(key, evt);
}
//Remove original .chukwa file
localFs.delete(originalPath,false);
//Rename the file from .recover back to .chukwa
localFs.rename(originalPath, newPath);
seqFileReader.close();
seqFileWriter.close();
newOutputStr.close();
}
catch(Exception e){
log.warn("Error while copying .chukwa file to valid sink
file",e);
e.printStackTrace();
}
}
}
However when the createValidSequenceFile method is called a ChecksumException
is thrown when seqFileReader.next(key, evt) is being executed. (BTW this is the
same exception that was causing moveFile to fail in LocalToRemoteHDFSMover, it
seems to happen when the reader reaches the corrupt chunk).
My question is : Is there a way to identify that the chunk being read is
corrupt and if so, not to read it, and close the sink file being copied to?
> Collectors don't finish writing .done datasink from last .chukwa datasink
> when stopped using bin/stop-collectors
> ----------------------------------------------------------------------------------------------------------------
>
> Key: CHUKWA-4
> URL: https://issues.apache.org/jira/browse/CHUKWA-4
> Project: Hadoop Chukwa
> Issue Type: Bug
> Components: data collection
> Environment: I am running on our local cluster. This is a linux
> machine that I also run Hadoop cluster from.
> Reporter: Andy Konwinski
> Priority: Minor
>
> When I use start-collectors, it creates the datasink as expected, writes to
> it as per normal, i.e. writes to the .chukwa file, and roll overs work fine
> when it renames the .chukwa file to .done. However, when I use
> bin/stop-collectors to shut down the running collector it leaves a .chukwa
> file in the HDFS file system. Not sure if this is a valid sink or not, but I
> think that the collector should gracefully clean up the datasink and rename
> it .done before exiting.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.