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 2007/06/11 22:38:46 UTC
svn commit: r546264 - in /lucene/hadoop/trunk: ./
src/java/org/apache/hadoop/filecache/ src/java/org/apache/hadoop/fs/
src/java/org/apache/hadoop/io/ src/java/org/apache/hadoop/util/
Author: cutting
Date: Mon Jun 11 13:38:44 2007
New Revision: 546264
URL: http://svn.apache.org/viewvc?view=rev&rev=546264
Log:
HADOOP-1412. Fix 'dodgy' bugs identified by FindBugs in fs and io packages. Contributed by Hairong.
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/DF.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/util/MergeSort.java
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Mon Jun 11 13:38:44 2007
@@ -84,8 +84,11 @@
27. HADOOP-1414. Fix a number of issues identified by FindBugs as
"Bad Practice". (Dhruba Borthakur via cutting)
- 28. HADOOP-1392. Fix "correctness" bugs reported by FindBugs in
+ 28. HADOOP-1392. Fix "correctness" bugs identified by FindBugs in
fs and dfs packages. (Raghu Angadi via cutting)
+
+ 29. HADOOP-1412. Fix "dodgy" bugs identified by FindBugs in fs and
+ io packages. (Hairong Kuang via cutting)
Release 0.13.0 - 2007-06-08
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java Mon Jun 11 13:38:44 2007
@@ -327,9 +327,7 @@
} else {
FSDataInputStream fsStream = fileSystem.open(md5File);
digest = new byte[md5.getDigestLength()];
- // assuming reading 16 bytes once is not a problem
- // though it should be checked if 16 bytes have been read or not
- int read = fsStream.read(digest);
+ fsStream.readFully(digest);
fsStream.close();
}
@@ -636,7 +634,7 @@
String frag1 = uriFiles[i].getFragment();
if (frag1 == null)
return false;
- for (int j=i+1; j < uriFiles.length; i++){
+ for (int j=i+1; j < uriFiles.length; j++){
String frag2 = uriFiles[j].getFragment();
if (frag2 == null)
return false;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/DF.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/DF.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/DF.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/DF.java Mon Jun 11 13:38:44 2007
@@ -144,12 +144,20 @@
private void parseExecResult(BufferedReader lines) throws IOException {
lines.readLine(); // skip headings
+ String line = lines.readLine();
+ if (line == null) {
+ throw new IOException( "Expecting a line not the end of stream" );
+ }
StringTokenizer tokens =
- new StringTokenizer(lines.readLine(), " \t\n\r\f%");
+ new StringTokenizer(line, " \t\n\r\f%");
this.filesystem = tokens.nextToken();
if (!tokens.hasMoreTokens()) { // for long filesystem name
- tokens = new StringTokenizer(lines.readLine(), " \t\n\r\f%");
+ line = lines.readLine();
+ if (line == null) {
+ throw new IOException( "Expecting a line not the end of stream" );
+ }
+ tokens = new StringTokenizer(line, " \t\n\r\f%");
}
this.capacity = Long.parseLong(tokens.nextToken()) * 1024;
this.used = Long.parseLong(tokens.nextToken()) * 1024;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java Mon Jun 11 13:38:44 2007
@@ -66,7 +66,7 @@
public void initialize(URI uri, Configuration conf) {
setConf(conf);
int size = Integer.parseInt(conf.get("fs.inmemory.size.mb", "100"));
- this.fsSize = size * 1024 * 1024;
+ this.fsSize = size * 1024L * 1024L;
this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
String path = this.uri.getPath();
if (path.length() == 0) {
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java Mon Jun 11 13:38:44 2007
@@ -387,7 +387,7 @@
int high = count-1;
while (low <= high) {
- int mid = (low + high) >> 1;
+ int mid = (low + high) >>> 1;
WritableComparable midVal = keys[mid];
int cmp = comparator.compare(midVal, key);
@@ -537,7 +537,6 @@
String out = args[1];
Configuration conf = new Configuration();
- int ioFileBufferSize = conf.getInt("io.file.buffer.size", 4096);
FileSystem fs = FileSystem.getLocal(conf);
MapFile.Reader reader = new MapFile.Reader(fs, in, conf);
MapFile.Writer writer =
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java Mon Jun 11 13:38:44 2007
@@ -1827,7 +1827,7 @@
int segments = sortPass(deleteInput);
if (segments > 1) {
- segments = mergePass(outFile.getParent());
+ mergePass(outFile.getParent());
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java Mon Jun 11 13:38:44 2007
@@ -571,17 +571,24 @@
reporter.setStatus("Copied: " + srcURI.toString() +
" to: " + destinationPath.toString());
- } catch(Exception e) {
- reporter.setStatus("Failed to copy from: " + (Text)key);
- if (ignoreReadFailures) {
- return;
- } else {
- throw new IOException("Failed to copy from: " + (Text)key);
- }
+ } catch (URISyntaxException e) {
+ handleException(reporter, (Text)key, e);
+ } catch (IOException ioe) {
+ handleException(reporter,(Text)key, ioe);
+ }
+ }
+
+ /* handle exceptions */
+ private void handleException( Reporter reporter, Text key, Throwable e )
+ throws IOException {
+ String errMsg = "Failed to copy from: " + (Text)key;
+ reporter.setStatus(errMsg);
+ if ( !ignoreReadFailures ) {
+ throw new IOException(errMsg);
}
}
}
-
+
/**
* Factory to create requisite Mapper objects for distcp.
* @author Arun C Murthy
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/MergeSort.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/MergeSort.java?view=diff&rev=546264&r1=546263&r2=546264
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/MergeSort.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/MergeSort.java Mon Jun 11 13:38:44 2007
@@ -51,7 +51,7 @@
}
// Recursively sort halves of dest into src
- int mid = (low + high) >> 1;
+ int mid = (low + high) >>> 1;
mergeSort(dest, src, low, mid);
mergeSort(dest, src, mid, high);