HDFS-9187. Fix null pointer error in Globber when FS was not constructed via FileSystem#createFileSystem (cmccabe)

This commit is contained in:
Colin Patrick Mccabe 2015-10-13 13:11:33 -07:00
parent da8441d0fe
commit d286032b71
3 changed files with 19 additions and 8 deletions

View File

@ -129,12 +129,6 @@ public abstract class FileSystem extends Configured implements Closeable {
boolean resolveSymlinks;
private Tracer tracer;
protected final Tracer getTracer() {
return tracer;
}
/**
* This method adds a file system for testing so that we can find it later. It
* is only for testing.
@ -2712,7 +2706,6 @@ private static FileSystem createFileSystem(URI uri, Configuration conf
try {
Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
fs.tracer = tracer;
fs.initialize(uri, conf);
return fs;
} finally {

View File

@ -47,7 +47,7 @@ public Globber(FileSystem fs, Path pathPattern, PathFilter filter) {
this.fc = null;
this.pathPattern = pathPattern;
this.filter = filter;
this.tracer = fs.getTracer();
this.tracer = FsTracer.get(fs.getConf());
}
public Globber(FileContext fc, Path pathPattern, PathFilter filter) {

View File

@ -18,8 +18,13 @@
package org.apache.hadoop.tracing;
import static org.junit.Assert.assertEquals;
import java.net.URI;
import java.util.LinkedList;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
import org.apache.htrace.core.HTraceConfiguration;
import org.junit.Test;
@ -50,4 +55,17 @@ public void testExtraConfig() {
HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(TEST_PREFIX, conf, extraConfig);
assertEquals(newValue, wrapped.get(key));
}
/**
* Test tracing the globber. This is a regression test for HDFS-9187.
*/
@Test
public void testTracingGlobber() throws Exception {
// Bypass the normal FileSystem object creation path by just creating an
// instance of a subclass.
FileSystem fs = new LocalFileSystem();
fs.initialize(new URI("file:///"), new Configuration());
fs.globStatus(new Path("/"));
fs.close();
}
}