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