You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/05/15 22:13:11 UTC
svn commit: r406732 - in /lucene/hadoop/trunk: CHANGES.txt
src/java/org/apache/hadoop/io/SequenceFile.java
Author: cutting
Date: Mon May 15 13:13:11 2006
New Revision: 406732
URL: http://svn.apache.org/viewcvs?rev=406732&view=rev
Log:
HADOOP-219. Fix a NPE when handling a checksum exception under SequenceFile.Sorter.sort(). With help from Stack.
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/CHANGES.txt?rev=406732&r1=406731&r2=406732&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Mon May 15 13:13:11 2006
@@ -26,6 +26,9 @@
henceforth refer to them by number when collecting map output.
(omalley via cutting)
+ 7. HADOOP-219. Fix a NullPointerException when handling a checksum
+ exception under SequenceFile.Sorter.sort(). (cutting & stack)
+
Release 0.2.1 - 2006-05-12
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java?rev=406732&r1=406731&r2=406732&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java Mon May 15 13:13:11 2006
@@ -235,23 +235,25 @@
/** Open the named file. */
public Reader(FileSystem fs, Path file, Configuration conf)
throws IOException {
- this(fs, file, conf.getInt("io.file.buffer.size", 4096));
- this.conf = conf;
+ this(fs, file, conf.getInt("io.file.buffer.size", 4096), conf);
}
- private Reader(FileSystem fs, Path name, int bufferSize) throws IOException {
+ private Reader(FileSystem fs, Path name, int bufferSize,
+ Configuration conf) throws IOException {
this.fs = fs;
this.file = name;
this.in = fs.open(file, bufferSize);
this.end = fs.getLength(file);
+ this.conf = conf;
init();
}
- private Reader(FileSystem fs, Path file, int bufferSize, long start, long length)
- throws IOException {
+ private Reader(FileSystem fs, Path file, int bufferSize, long start,
+ long length, Configuration conf) throws IOException {
this.fs = fs;
this.file = file;
this.in = fs.open(file, bufferSize);
+ this.conf = conf;
seek(start);
init();
@@ -742,7 +744,7 @@
totalCount+= count;
Reader reader = new Reader(fs, inName, memory/(factor+1),
- in.getPos(), length);
+ in.getPos(), length, conf);
reader.sync = null; // disable sync on temp files
MergeStream ms = new MergeStream(reader); // add segment to queue
@@ -801,7 +803,7 @@
for (int i = 0; i < inFiles.length; i++) {
Path inFile = inFiles[i];
MergeStream ms =
- new MergeStream(new Reader(fs, inFile, memory/(factor+1)));
+ new MergeStream(new Reader(fs, inFile, memory/(factor+1), conf));
if (ms.next())
queue.put(ms);
}