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