001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.zookeeper;
020
021import java.io.File;
022import java.io.IOException;
023import java.io.PrintWriter;
024import java.net.InetAddress;
025import java.net.NetworkInterface;
026import java.nio.charset.StandardCharsets;
027import java.util.ArrayList;
028import java.util.Enumeration;
029import java.util.List;
030import java.util.Map.Entry;
031import java.util.Properties;
032
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.hbase.HBaseConfiguration;
035import org.apache.hadoop.hbase.HBaseInterfaceAudience;
036import org.apache.hadoop.hbase.HConstants;
037import org.apache.hadoop.hbase.util.DNS;
038import org.apache.hadoop.hbase.util.Strings;
039import org.apache.hadoop.util.StringUtils;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.apache.yetus.audience.InterfaceStability;
042import org.apache.zookeeper.server.ServerConfig;
043import org.apache.zookeeper.server.ZooKeeperServerMain;
044import org.apache.zookeeper.server.admin.AdminServer;
045import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
046import org.apache.zookeeper.server.quorum.QuorumPeerMain;
047import org.apache.zookeeper.server.DatadirCleanupManager;
048import org.slf4j.Logger;
049import org.slf4j.LoggerFactory;
050
051/**
052 * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
053 * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
054 * things in here rather than directly calling to ZooKeeper, we have more
055 * control over the process. This class uses {@link ZKConfig} to get settings
056 * from the hbase-site.xml file.
057 */
058@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
059@InterfaceStability.Evolving
060public final class HQuorumPeer {
061  private HQuorumPeer() {
062  }
063
064  /**
065   * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
066   * @param args String[] of command line arguments. Not used.
067   */
068  public static void main(String[] args) {
069    Configuration conf = HBaseConfiguration.create();
070    try {
071      Properties zkProperties = ZKConfig.makeZKProps(conf);
072      writeMyID(zkProperties);
073      QuorumPeerConfig zkConfig = new QuorumPeerConfig();
074      zkConfig.parseProperties(zkProperties);
075
076      // login the zookeeper server principal (if using security)
077      ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE,
078        HConstants.ZK_SERVER_KERBEROS_PRINCIPAL,
079        zkConfig.getClientPortAddress().getHostName());
080
081      runZKServer(zkConfig);
082    } catch (Exception e) {
083      e.printStackTrace();
084      System.exit(-1);
085    }
086  }
087
088  private static void runZKServer(QuorumPeerConfig zkConfig)
089          throws IOException, AdminServer.AdminServerException {
090
091    /**
092     *  Start and schedule the purge task
093     *  autopurge.purgeInterval is 0 by default,so in fact the DatadirCleanupManager task will not
094     *  be started to clean the logs by default. Config is recommended only for standalone server.
095     */
096
097    DatadirCleanupManager purgeMgr=new DatadirCleanupManager(
098      zkConfig.getDataDir(),
099      zkConfig.getDataLogDir(),
100      zkConfig.getSnapRetainCount(),
101      zkConfig.getPurgeInterval());
102    purgeMgr.start();
103
104    if (zkConfig.isDistributed()) {
105      QuorumPeerMain qp = new QuorumPeerMain();
106      qp.runFromConfig(zkConfig);
107    } else {
108      ZooKeeperServerMain zk = new ZooKeeperServerMain();
109      ServerConfig serverConfig = new ServerConfig();
110      serverConfig.readFrom(zkConfig);
111      zk.runFromConfig(serverConfig);
112    }
113  }
114
115  private static boolean addressIsLocalHost(String address) {
116    return address.equals("localhost") || address.equals("127.0.0.1");
117  }
118
119  static void writeMyID(Properties properties) throws IOException {
120    long myId = -1;
121
122    Configuration conf = HBaseConfiguration.create();
123    String myAddress = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
124        conf.get("hbase.zookeeper.dns.interface","default"),
125        conf.get("hbase.zookeeper.dns.nameserver","default")));
126
127    List<String> ips = new ArrayList<>();
128
129    // Add what could be the best (configured) match
130    ips.add(myAddress.contains(".") ?
131        myAddress :
132        StringUtils.simpleHostname(myAddress));
133
134    // For all nics get all hostnames and IPs
135    Enumeration<?> nics = NetworkInterface.getNetworkInterfaces();
136    while(nics.hasMoreElements()) {
137      Enumeration<?> rawAdrs =
138          ((NetworkInterface)nics.nextElement()).getInetAddresses();
139      while(rawAdrs.hasMoreElements()) {
140        InetAddress inet = (InetAddress) rawAdrs.nextElement();
141        ips.add(StringUtils.simpleHostname(inet.getHostName()));
142        ips.add(inet.getHostAddress());
143      }
144    }
145
146    for (Entry<Object, Object> entry : properties.entrySet()) {
147      String key = entry.getKey().toString().trim();
148      String value = entry.getValue().toString().trim();
149      if (key.startsWith("server.")) {
150        int dot = key.indexOf('.');
151        long id = Long.parseLong(key.substring(dot + 1));
152        String[] parts = value.split(":");
153        String address = parts[0];
154        if (addressIsLocalHost(address) || ips.contains(address)) {
155          myId = id;
156          break;
157        }
158      }
159    }
160
161    // Set the max session timeout from the provided client-side timeout
162    properties.setProperty("maxSessionTimeout", conf.get(HConstants.ZK_SESSION_TIMEOUT,
163            Integer.toString(HConstants.DEFAULT_ZK_SESSION_TIMEOUT)));
164
165    if (myId == -1) {
166      throw new IOException("Could not find my address: " + myAddress +
167                            " in list of ZooKeeper quorum servers");
168    }
169
170    String dataDirStr = properties.get("dataDir").toString().trim();
171    File dataDir = new File(dataDirStr);
172    if (!dataDir.isDirectory()) {
173      if (!dataDir.mkdirs()) {
174        throw new IOException("Unable to create data dir " + dataDir);
175      }
176    }
177
178    File myIdFile = new File(dataDir, "myid");
179    PrintWriter w = new PrintWriter(myIdFile, StandardCharsets.UTF_8.name());
180    w.println(myId);
181    w.close();
182  }
183}