Author: cutting Date: Tue Jul 26 09:40:00 2005 New Revision: 225344 URL: http://svn.apache.org/viewcvs?rev=225344&view=rev Log: Fix bug with syncs in large merges.
Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/io/SequenceFile.java Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/io/SequenceFile.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/io/SequenceFile.java?rev=225344&r1=225343&r2=225344&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/io/SequenceFile.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/io/SequenceFile.java Tue Jul 26 09:40:00 2005 @@ -618,7 +618,8 @@ this.pass = pass; this.last = last; - this.queue = new MergeQueue(factor, last ? outFile : outFile+"."+pass); + this.queue = + new MergeQueue(factor, last ? outFile : outFile+"."+pass, last); this.inName = outFile+"."+(pass-1); this.in = new NFSDataInputStream(nfs.open(new File(inName))); @@ -695,7 +696,7 @@ private MergeQueue queue; public MergeFiles() throws IOException { - this.queue = new MergeQueue(factor, outFile); + this.queue = new MergeQueue(factor, outFile, true); } public void close() throws IOException { @@ -741,12 +742,15 @@ private class MergeQueue extends PriorityQueue { private NFSDataOutputStream out; + private boolean done; - public MergeQueue(int size, String outName) throws IOException { + public MergeQueue(int size, String outName, boolean done) + throws IOException { initialize(size); this.out = new NFSDataOutputStream(nfs.create(new File(outName)), memory/(factor+1)); + this.done = done; } protected boolean lessThan(Object a, Object b) { @@ -758,6 +762,9 @@ public void merge() throws IOException { Writer writer = new Writer(out, keyClass, valClass); + if (!done) { + writer.sync = null; // disable sync on temp files + } while (size() != 0) { MergeStream ms = (MergeStream)top();