diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java index 498b95c..34cf495 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java @@ -20,11 +20,17 @@ package org.apache.hadoop.hbase.master; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DaemonThreadFactory; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.net.DNSToSwitchMapping; @@ -40,7 +46,10 @@ public class RackManager { public static final String UNKNOWN_RACK = "Unknown Rack"; private DNSToSwitchMapping switchMapping; - + private final int numThreads = 3; + private final ExecutorService executorService = + Executors.newFixedThreadPool(numThreads, new DaemonThreadFactory("rackmanager")); + public RackManager() { } @@ -57,15 +66,31 @@ public class RackManager { * @param server the server for which to get the rack name * @return the rack name of the server */ - public String getRack(ServerName server) { + public String getRack(final ServerName server) { if (server == null) { return UNKNOWN_RACK; } - // just a note - switchMapping caches results (at least the implementation should unless the - // resolution is really a lightweight process) - List racks = switchMapping.resolve(Arrays.asList(server.getHostname())); - if (racks != null && !racks.isEmpty()) { - return racks.get(0); + final List racks = new ArrayList(); + final String hostname = server.getHostname(); + Future f = (Future)executorService.submit(new Runnable() { + + @Override + public void run() { + // just a note - switchMapping caches results (at least the implementation should unless the + // resolution is really a lightweight process) + racks.addAll(switchMapping.resolve(Arrays.asList(hostname))); + } + }); + try { + f.get(60000, TimeUnit.MILLISECONDS); + if (racks != null && !racks.isEmpty()) { + return racks.get(0); + } + } catch (TimeoutException e) { + f.cancel(true); + LOG.warn("Timeout retrieving rack for " + hostname); + } catch (Exception e) { + LOG.warn("Got exception retrieving rack for " + hostname, e); } return UNKNOWN_RACK;