1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.zookeeper;
21
22 import java.io.IOException;
23 import java.io.InputStream;
24 import java.net.InetAddress;
25 import java.net.UnknownHostException;
26 import java.util.ArrayList;
27 import java.util.List;
28 import java.util.Properties;
29 import java.util.Map.Entry;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.util.StringUtils;
36
37
38
39
40 public class ZKConfig {
41 private static final Log LOG = LogFactory.getLog(ZKConfig.class);
42
43 private static final String VARIABLE_START = "${";
44 private static final int VARIABLE_START_LENGTH = VARIABLE_START.length();
45 private static final String VARIABLE_END = "}";
46 private static final int VARIABLE_END_LENGTH = VARIABLE_END.length();
47
48
49
50
51
52
53
54
55
56 public static Properties makeZKProps(Configuration conf) {
57
58
59 ClassLoader cl = HQuorumPeer.class.getClassLoader();
60 final InputStream inputStream =
61 cl.getResourceAsStream(HConstants.ZOOKEEPER_CONFIG_NAME);
62 if (inputStream != null) {
63 try {
64 return parseZooCfg(conf, inputStream);
65 } catch (IOException e) {
66 LOG.warn("Cannot read " + HConstants.ZOOKEEPER_CONFIG_NAME +
67 ", loading from XML files", e);
68 }
69 }
70
71
72 Properties zkProperties = new Properties();
73
74
75 for (Entry<String, String> entry : conf) {
76 String key = entry.getKey();
77 if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
78 String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN);
79 String value = entry.getValue();
80
81 if (value.contains(VARIABLE_START)) {
82 value = conf.get(key);
83 }
84 zkProperties.put(zkKey, value);
85 }
86 }
87
88
89 if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) {
90 zkProperties.put(HConstants.CLIENT_PORT_STR,
91 HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
92 }
93
94
95 int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
96 int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
97
98 final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
99 HConstants.LOCALHOST);
100 for (int i = 0; i < serverHosts.length; ++i) {
101 String serverHost = serverHosts[i];
102 String address = serverHost + ":" + peerPort + ":" + leaderPort;
103 String key = "server." + i;
104 zkProperties.put(key, address);
105 }
106
107 return zkProperties;
108 }
109
110
111
112
113
114
115
116
117
118 public static Properties parseZooCfg(Configuration conf,
119 InputStream inputStream) throws IOException {
120 Properties properties = new Properties();
121 try {
122 properties.load(inputStream);
123 } catch (IOException e) {
124 final String msg = "fail to read properties from "
125 + HConstants.ZOOKEEPER_CONFIG_NAME;
126 LOG.fatal(msg);
127 throw new IOException(msg, e);
128 }
129 for (Entry<Object, Object> entry : properties.entrySet()) {
130 String value = entry.getValue().toString().trim();
131 String key = entry.getKey().toString().trim();
132 StringBuilder newValue = new StringBuilder();
133 int varStart = value.indexOf(VARIABLE_START);
134 int varEnd = 0;
135 while (varStart != -1) {
136 varEnd = value.indexOf(VARIABLE_END, varStart);
137 if (varEnd == -1) {
138 String msg = "variable at " + varStart + " has no end marker";
139 LOG.fatal(msg);
140 throw new IOException(msg);
141 }
142 String variable = value.substring(varStart + VARIABLE_START_LENGTH, varEnd);
143
144 String substituteValue = System.getProperty(variable);
145 if (substituteValue == null) {
146 substituteValue = conf.get(variable);
147 }
148 if (substituteValue == null) {
149 String msg = "variable " + variable + " not set in system property "
150 + "or hbase configs";
151 LOG.fatal(msg);
152 throw new IOException(msg);
153 }
154
155 newValue.append(substituteValue);
156
157 varEnd += VARIABLE_END_LENGTH;
158 varStart = value.indexOf(VARIABLE_START, varEnd);
159 }
160
161 if (key.startsWith("server.")) {
162 boolean mode = conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED);
163 if (mode == HConstants.CLUSTER_IS_DISTRIBUTED && value.startsWith(HConstants.LOCALHOST)) {
164 String msg = "The server in zoo.cfg cannot be set to localhost " +
165 "in a fully-distributed setup because it won't be reachable. " +
166 "See \"Getting Started\" for more information.";
167 LOG.fatal(msg);
168 throw new IOException(msg);
169 }
170 }
171 newValue.append(value.substring(varEnd));
172 properties.setProperty(key, newValue.toString());
173 }
174 return properties;
175 }
176
177
178
179
180
181
182
183 public static String getZKQuorumServersString(Properties properties) {
184 String clientPort = null;
185 List<String> servers = new ArrayList<String>();
186
187
188
189 boolean anyValid = false;
190 for (Entry<Object,Object> property : properties.entrySet()) {
191 String key = property.getKey().toString().trim();
192 String value = property.getValue().toString().trim();
193 if (key.equals("clientPort")) {
194 clientPort = value;
195 }
196 else if (key.startsWith("server.")) {
197 String host = value.substring(0, value.indexOf(':'));
198 servers.add(host);
199 try {
200
201 InetAddress.getByName(host);
202 anyValid = true;
203 } catch (UnknownHostException e) {
204 LOG.warn(StringUtils.stringifyException(e));
205 }
206 }
207 }
208
209 if (!anyValid) {
210 LOG.error("no valid quorum servers found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
211 return null;
212 }
213
214 if (clientPort == null) {
215 LOG.error("no clientPort found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
216 return null;
217 }
218
219 if (servers.isEmpty()) {
220 LOG.fatal("No server.X lines found in conf/zoo.cfg. HBase must have a " +
221 "ZooKeeper cluster configured for its operation.");
222 return null;
223 }
224
225 StringBuilder hostPortBuilder = new StringBuilder();
226 for (int i = 0; i < servers.size(); ++i) {
227 String host = servers.get(i);
228 if (i > 0) {
229 hostPortBuilder.append(',');
230 }
231 hostPortBuilder.append(host);
232 hostPortBuilder.append(':');
233 hostPortBuilder.append(clientPort);
234 }
235
236 return hostPortBuilder.toString();
237 }
238
239
240
241
242
243
244 public static String getZKQuorumServersString(Configuration conf) {
245 return getZKQuorumServersString(makeZKProps(conf));
246 }
247 }