1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.master;
19
20 import java.util.ArrayList;
21 import java.util.Arrays;
22 import java.util.List;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.hbase.ServerName;
29 import org.apache.hadoop.hbase.util.ReflectionUtils;
30 import org.apache.hadoop.net.DNSToSwitchMapping;
31 import org.apache.hadoop.net.ScriptBasedMapping;
32
33
34
35
36
37 @InterfaceAudience.Private
38 public class RackManager {
39 private static final Log LOG = LogFactory.getLog(RackManager.class);
40 public static final String UNKNOWN_RACK = "Unknown Rack";
41
42 private DNSToSwitchMapping switchMapping;
43
44 public RackManager() {
45 }
46
47 public RackManager(Configuration conf) {
48 switchMapping = ReflectionUtils.instantiateWithCustomCtor(
49 conf.getClass("hbase.util.ip.to.rack.determiner", ScriptBasedMapping.class,
50 DNSToSwitchMapping.class).getName(), new Class<?>[]{Configuration.class},
51 new Object[]{conf});
52 }
53
54
55
56
57
58
59
60 public String getRack(ServerName server) {
61 if (server == null) {
62 return UNKNOWN_RACK;
63 }
64
65
66 List<String> racks = switchMapping.resolve(Arrays.asList(server.getHostname()));
67 if (racks != null && !racks.isEmpty()) {
68 return racks.get(0);
69 }
70
71 return UNKNOWN_RACK;
72 }
73
74
75
76
77
78
79 public List<String> getRack(List<ServerName> servers) {
80
81
82 List<String> serversAsString = new ArrayList<String>(servers.size());
83 for (ServerName server : servers) {
84 serversAsString.add(server.getHostname());
85 }
86 List<String> racks = switchMapping.resolve(serversAsString);
87 return racks;
88 }
89 }