|
@@ -17,7 +17,6 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
-import java.io.Closeable;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.InetAddress;
|
|
import java.net.InetAddress;
|
|
import java.net.UnknownHostException;
|
|
import java.net.UnknownHostException;
|
|
@@ -26,18 +25,12 @@ import java.util.HashSet;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.TreeMap;
|
|
import java.util.TreeMap;
|
|
-import java.util.concurrent.ScheduledFuture;
|
|
|
|
-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.util.HostsFileReader;
|
|
import org.apache.hadoop.util.HostsFileReader;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* This class manages the include and exclude files for HDFS.
|
|
* This class manages the include and exclude files for HDFS.
|
|
*
|
|
*
|
|
@@ -74,12 +67,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
* The "registration name" feature is a little odd and it may be removed in the
|
|
* The "registration name" feature is a little odd and it may be removed in the
|
|
* future (I hope?)
|
|
* future (I hope?)
|
|
*/
|
|
*/
|
|
-public class HostFileManager implements Closeable {
|
|
|
|
|
|
+public class HostFileManager {
|
|
private static final Log LOG = LogFactory.getLog(HostFileManager.class);
|
|
private static final Log LOG = LogFactory.getLog(HostFileManager.class);
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public static boolean dnsResolutionDisabledForTesting = false;
|
|
|
|
-
|
|
|
|
public static class Entry {
|
|
public static class Entry {
|
|
/**
|
|
/**
|
|
* This what the user put on the line before the colon, or the whole line
|
|
* This what the user put on the line before the colon, or the whole line
|
|
@@ -98,38 +88,6 @@ public class HostFileManager implements Closeable {
|
|
* empty string.
|
|
* empty string.
|
|
*/
|
|
*/
|
|
private final String ipAddress;
|
|
private final String ipAddress;
|
|
-
|
|
|
|
- public Entry(String prefix, int port, String ipAddress) {
|
|
|
|
- this.prefix = prefix;
|
|
|
|
- this.port = port;
|
|
|
|
- this.ipAddress = ipAddress;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public String getIdentifier() {
|
|
|
|
- return ipAddress.isEmpty() ? prefix : ipAddress;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- static Entry fromPrefixAndPort(String fileName, String prefix, int port) {
|
|
|
|
- String ipAddress = "";
|
|
|
|
- try {
|
|
|
|
- if (dnsResolutionDisabledForTesting) {
|
|
|
|
- throw new UnknownHostException("dns resolution disabled for " +
|
|
|
|
- "testing");
|
|
|
|
- }
|
|
|
|
- // Let's see if we can resolve this prefix to an IP address.
|
|
|
|
- // This may fail; one example is with a registered hostname
|
|
|
|
- // which is not actually a real DNS name.
|
|
|
|
- InetAddress addr = InetAddress.getByName(prefix);
|
|
|
|
- ipAddress = addr.getHostAddress();
|
|
|
|
- } catch (UnknownHostException e) {
|
|
|
|
- if (fileName != null) {
|
|
|
|
- LOG.info("When reading " + fileName + ", could not look up " +
|
|
|
|
- "IP address for " + prefix + ". We will assume this is a " +
|
|
|
|
- "registration name.", e);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return new Entry(prefix, port, ipAddress);
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Parse a hosts file Entry.
|
|
* Parse a hosts file Entry.
|
|
@@ -137,6 +95,7 @@ public class HostFileManager implements Closeable {
|
|
static Entry parse(String fileName, String entry) throws IOException {
|
|
static Entry parse(String fileName, String entry) throws IOException {
|
|
final String prefix;
|
|
final String prefix;
|
|
final int port;
|
|
final int port;
|
|
|
|
+ String ipAddress = "";
|
|
|
|
|
|
int idx = entry.indexOf(':');
|
|
int idx = entry.indexOf(':');
|
|
if (-1 == idx) {
|
|
if (-1 == idx) {
|
|
@@ -152,7 +111,28 @@ public class HostFileManager implements Closeable {
|
|
"'" + entry + "'", e);
|
|
"'" + entry + "'", e);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- return Entry.fromPrefixAndPort(fileName, prefix, port);
|
|
|
|
|
|
+ try {
|
|
|
|
+ // Let's see if we can resolve this prefix to an IP address.
|
|
|
|
+ // This may fail; one example is with a registered hostname
|
|
|
|
+ // which is not actually a real DNS name.
|
|
|
|
+ InetAddress addr = InetAddress.getByName(prefix);
|
|
|
|
+ ipAddress = addr.getHostAddress();
|
|
|
|
+ } catch (UnknownHostException e) {
|
|
|
|
+ LOG.info("When reading " + fileName + ", could not look up " +
|
|
|
|
+ "IP address for " + prefix + ". We will assume this is a " +
|
|
|
|
+ "registration name.", e);
|
|
|
|
+ }
|
|
|
|
+ return new Entry(prefix, port, ipAddress);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public String getIdentifier() {
|
|
|
|
+ return ipAddress.isEmpty() ? prefix : ipAddress;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Entry(String prefix, int port, String ipAddress) {
|
|
|
|
+ this.prefix = prefix;
|
|
|
|
+ this.port = port;
|
|
|
|
+ this.ipAddress = ipAddress;
|
|
}
|
|
}
|
|
|
|
|
|
public String getPrefix() {
|
|
public String getPrefix() {
|
|
@@ -293,47 +273,7 @@ public class HostFileManager implements Closeable {
|
|
private EntrySet includes = new EntrySet();
|
|
private EntrySet includes = new EntrySet();
|
|
private EntrySet excludes = new EntrySet();
|
|
private EntrySet excludes = new EntrySet();
|
|
|
|
|
|
- private final ScheduledThreadPoolExecutor executor
|
|
|
|
- = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
|
|
|
- setDaemon(true).setNameFormat("HostFileManagerDnsRefresh thread").
|
|
|
|
- build());
|
|
|
|
-
|
|
|
|
- private final ScheduledFuture<?> dnsResolverFuture;
|
|
|
|
-
|
|
|
|
- private class DnsResolver implements Runnable {
|
|
|
|
- @Override
|
|
|
|
- public void run() {
|
|
|
|
- EntrySet oldIncludes, oldExcludes;
|
|
|
|
- synchronized (HostFileManager.this) {
|
|
|
|
- oldIncludes = includes;
|
|
|
|
- oldExcludes = excludes;
|
|
|
|
- }
|
|
|
|
- MutableEntrySet newIncludes = new MutableEntrySet();
|
|
|
|
- for (Entry e : oldIncludes) {
|
|
|
|
- newIncludes.add(Entry.fromPrefixAndPort(null, e.prefix, e.port));
|
|
|
|
- }
|
|
|
|
- MutableEntrySet newExcludes = new MutableEntrySet();
|
|
|
|
- for (Entry e : oldExcludes) {
|
|
|
|
- newExcludes.add(Entry.fromPrefixAndPort(null, e.prefix, e.port));
|
|
|
|
- }
|
|
|
|
- synchronized (HostFileManager.this) {
|
|
|
|
- // Don't replace an entry set that has already been replaced by
|
|
|
|
- // refresh().
|
|
|
|
- if (includes == oldIncludes) {
|
|
|
|
- includes = newIncludes;
|
|
|
|
- }
|
|
|
|
- if (excludes == oldExcludes) {
|
|
|
|
- excludes = newExcludes;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public HostFileManager(int dnsResolutionSeconds) {
|
|
|
|
- this.dnsResolverFuture = this.executor.
|
|
|
|
- scheduleAtFixedRate(new DnsResolver(),
|
|
|
|
- dnsResolutionSeconds, dnsResolutionSeconds,
|
|
|
|
- TimeUnit.SECONDS);
|
|
|
|
|
|
+ public HostFileManager() {
|
|
}
|
|
}
|
|
|
|
|
|
public void refresh(String includeFile, String excludeFile)
|
|
public void refresh(String includeFile, String excludeFile)
|
|
@@ -415,9 +355,4 @@ public class HostFileManager implements Closeable {
|
|
public synchronized EntrySet getExcludes() {
|
|
public synchronized EntrySet getExcludes() {
|
|
return excludes;
|
|
return excludes;
|
|
}
|
|
}
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public synchronized void close() throws IOException {
|
|
|
|
- dnsResolverFuture.cancel(false);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|