You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2015/05/20 01:33:12 UTC

hbase git commit: HBASE-13636 Remove deprecation for HBASE-4072 (Reading of zoo.cfg) Signed-off-by: stack

Repository: hbase
Updated Branches:
  refs/heads/master 02fa2200b -> 1fcee8697


HBASE-13636 Remove deprecation for HBASE-4072 (Reading of zoo.cfg)
Signed-off-by: stack <st...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1fcee869
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1fcee869
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1fcee869

Branch: refs/heads/master
Commit: 1fcee86978b20ac82647b5213eae389a38735cce
Parents: 02fa220
Author: Lars Francke <la...@gmail.com>
Authored: Wed May 6 19:06:59 2015 +0200
Committer: stack <st...@apache.org>
Committed: Tue May 19 16:32:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/zookeeper/HQuorumPeer.java     |   4 +-
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java | 203 +------------------
 .../org/apache/hadoop/hbase/HConstants.java     |   7 -
 .../src/main/resources/hbase-default.xml        |  10 +-
 hbase-rest/src/test/resources/zoo.cfg           |  43 ----
 hbase-server/pom.xml                            |   1 -
 .../hadoop/hbase/zookeeper/ZKServerTool.java    |  23 +--
 .../hadoop/hbase/zookeeper/TestHQuorumPeer.java |  58 +-----
 .../hadoop/hbase/zookeeper/TestZKConfig.java    |  28 +--
 hbase-server/src/test/resources/zoo.cfg         |  43 ----
 hbase-shell/pom.xml                             |   1 -
 pom.xml                                         |   1 -
 src/main/asciidoc/_chapters/hbase-default.adoc  |  15 --
 src/main/asciidoc/_chapters/zookeeper.adoc      |  27 +--
 14 files changed, 44 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
index 1e04948..2ad1f33 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
@@ -50,8 +50,8 @@ import org.apache.zookeeper.server.quorum.QuorumPeerMain;
  * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
  * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
  * things in here rather than directly calling to ZooKeeper, we have more
- * control over the process. This class uses {@link ZKConfig} to parse the
- * zoo.cfg and inject variables from HBase's site.xml configuration in.
+ * control over the process. This class uses {@link ZKConfig} to get settings
+ * from the hbase-site.xml file.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
index 7cf4b8d..a8f1182 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
@@ -18,15 +18,9 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map.Entry;
 import java.util.Properties;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -35,18 +29,13 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Utility methods for reading, and building the ZooKeeper configuration.
  *
  * The order and priority for reading the config are as follows:
- * (1). zoo.cfg if ""hbase.config.read.zookeeper.config" is true
- * (2). Property with "hbase.zookeeper.property." prefix from HBase XML
- * (3). other zookeeper related properties in HBASE XML
+ * (1). Property with "hbase.zookeeper.property." prefix from HBase XML
+ * (2). other zookeeper related properties in HBASE XML
  */
 @InterfaceAudience.Private
 public class ZKConfig {
-  private static final Log LOG = LogFactory.getLog(ZKConfig.class);
 
   private static final String VARIABLE_START = "${";
-  private static final int VARIABLE_START_LENGTH = VARIABLE_START.length();
-  private static final String VARIABLE_END = "}";
-  private static final int VARIABLE_END_LENGTH = VARIABLE_END.length();
 
   /**
    * Make a Properties object holding ZooKeeper config.
@@ -56,54 +45,7 @@ public class ZKConfig {
    * @return Properties holding mappings representing ZooKeeper config file.
    */
   public static Properties makeZKProps(Configuration conf) {
-    Properties zkProperties = makeZKPropsFromZooCfg(conf);
-
-    if (zkProperties == null) {
-      // Otherwise, use the configuration options from HBase's XML files.
-      zkProperties = makeZKPropsFromHbaseConfig(conf);
-    }
-    return zkProperties;
-  }
-
-  /**
-   * Parses the corresponding config options from the zoo.cfg file
-   * and make a Properties object holding the Zookeeper config.
-   *
-   * @param conf Configuration to read from.
-   * @return Properties holding mappings representing the ZooKeeper config file or null if
-   * the HBASE_CONFIG_READ_ZOOKEEPER_CONFIG is false or the file does not exist.
-   */
-  private static Properties makeZKPropsFromZooCfg(Configuration conf) {
-    if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, false)) {
-      LOG.warn(
-          "Parsing ZooKeeper's " + HConstants.ZOOKEEPER_CONFIG_NAME +
-          " file for ZK properties " +
-          "has been deprecated. Please instead place all ZK related HBase " +
-          "configuration under the hbase-site.xml, using prefixes " +
-          "of the form '" + HConstants.ZK_CFG_PROPERTY_PREFIX + "', and " +
-          "set property '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG +
-          "' to false");
-      // First check if there is a zoo.cfg in the CLASSPATH. If so, simply read
-      // it and grab its configuration properties.
-      ClassLoader cl = HQuorumPeer.class.getClassLoader();
-      final InputStream inputStream =
-        cl.getResourceAsStream(HConstants.ZOOKEEPER_CONFIG_NAME);
-      if (inputStream != null) {
-        try {
-          return parseZooCfg(conf, inputStream);
-        } catch (IOException e) {
-          LOG.warn("Cannot read " + HConstants.ZOOKEEPER_CONFIG_NAME +
-                   ", loading from XML files", e);
-        }
-      }
-    } else {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Skipped reading ZK properties file '" + HConstants.ZOOKEEPER_CONFIG_NAME +
-          "' since '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG + "' was not set to true");
-      }
-    }
-
-    return null;
+    return makeZKPropsFromHbaseConfig(conf);
   }
 
   /**
@@ -129,7 +71,7 @@ public class ZKConfig {
           if (value.contains(VARIABLE_START)) {
             value = conf.get(key);
           }
-          zkProperties.put(zkKey, value);
+          zkProperties.setProperty(zkKey, value);
         }
       }
     }
@@ -164,135 +106,6 @@ public class ZKConfig {
   }
 
   /**
-   * Parse ZooKeeper's zoo.cfg, injecting HBase Configuration variables in.
-   * This method is used for testing so we can pass our own InputStream.
-   * @param conf HBaseConfiguration to use for injecting variables.
-   * @param inputStream InputStream to read from.
-   * @return Properties parsed from config stream with variables substituted.
-   * @throws IOException if anything goes wrong parsing config
-   * @deprecated in 0.96 onwards. HBase will no longer rely on zoo.cfg
-   * availability.
-   */
-  @Deprecated
-  public static Properties parseZooCfg(Configuration conf,
-      InputStream inputStream) throws IOException {
-    Properties properties = new Properties();
-    try {
-      properties.load(inputStream);
-    } catch (IOException e) {
-      final String msg = "fail to read properties from "
-        + HConstants.ZOOKEEPER_CONFIG_NAME;
-      LOG.fatal(msg);
-      throw new IOException(msg, e);
-    }
-    for (Entry<Object, Object> entry : properties.entrySet()) {
-      String value = entry.getValue().toString().trim();
-      String key = entry.getKey().toString().trim();
-      StringBuilder newValue = new StringBuilder();
-      int varStart = value.indexOf(VARIABLE_START);
-      int varEnd = 0;
-      while (varStart != -1) {
-        varEnd = value.indexOf(VARIABLE_END, varStart);
-        if (varEnd == -1) {
-          String msg = "variable at " + varStart + " has no end marker";
-          LOG.fatal(msg);
-          throw new IOException(msg);
-        }
-        String variable = value.substring(varStart + VARIABLE_START_LENGTH, varEnd);
-
-        String substituteValue = System.getProperty(variable);
-        if (substituteValue == null) {
-          substituteValue = conf.get(variable);
-        }
-        if (substituteValue == null) {
-          String msg = "variable " + variable + " not set in system property "
-                     + "or hbase configs";
-          LOG.fatal(msg);
-          throw new IOException(msg);
-        }
-
-        newValue.append(substituteValue);
-
-        varEnd += VARIABLE_END_LENGTH;
-        varStart = value.indexOf(VARIABLE_START, varEnd);
-      }
-      // Special case for 'hbase.cluster.distributed' property being 'true'
-      if (key.startsWith("server.")) {
-        boolean mode =
-            conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED);
-        if (mode == HConstants.CLUSTER_IS_DISTRIBUTED && value.startsWith(HConstants.LOCALHOST)) {
-          String msg = "The server in zoo.cfg cannot be set to localhost " +
-              "in a fully-distributed setup because it won't be reachable. " +
-              "See \"Getting Started\" for more information.";
-          LOG.fatal(msg);
-          throw new IOException(msg);
-        }
-      }
-      newValue.append(value.substring(varEnd));
-      properties.setProperty(key, newValue.toString());
-    }
-    return properties;
-  }
-
-  /**
-   * Return the ZK Quorum servers string given zk properties returned by
-   * makeZKProps
-   * @param properties
-   * @return Quorum servers String
-   */
-  private static String getZKQuorumServersString(Properties properties) {
-    String clientPort = null;
-    List<String> servers = new ArrayList<String>();
-
-    // The clientPort option may come after the server.X hosts, so we need to
-    // grab everything and then create the final host:port comma separated list.
-    boolean anyValid = false;
-    for (Entry<Object,Object> property : properties.entrySet()) {
-      String key = property.getKey().toString().trim();
-      String value = property.getValue().toString().trim();
-      if (key.equals("clientPort")) {
-        clientPort = value;
-      }
-      else if (key.startsWith("server.")) {
-        String host = value.substring(0, value.indexOf(':'));
-        servers.add(host);
-        anyValid = true;
-      }
-    }
-
-    if (!anyValid) {
-      LOG.error("no valid quorum servers found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
-      return null;
-    }
-
-    if (clientPort == null) {
-      LOG.error("no clientPort found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
-      return null;
-    }
-
-    if (servers.isEmpty()) {
-      LOG.fatal("No servers were found in provided ZooKeeper configuration. " +
-          "HBase must have a ZooKeeper cluster configured for its " +
-          "operation. Ensure that you've configured '" +
-          HConstants.ZOOKEEPER_QUORUM + "' properly.");
-      return null;
-    }
-
-    StringBuilder hostPortBuilder = new StringBuilder();
-    for (int i = 0; i < servers.size(); ++i) {
-      String host = servers.get(i);
-      if (i > 0) {
-        hostPortBuilder.append(',');
-      }
-      hostPortBuilder.append(host);
-      hostPortBuilder.append(':');
-      hostPortBuilder.append(clientPort);
-    }
-
-    return hostPortBuilder.toString();
-  }
-
-  /**
    * Return the ZK Quorum servers string given the specified configuration
    *
    * @param conf
@@ -334,17 +147,9 @@ public class ZKConfig {
 
   /**
    * Return the ZK Quorum servers string given the specified configuration.
-   * @param conf
    * @return Quorum servers
    */
   public static String getZKQuorumServersString(Configuration conf) {
-    // First try zoo.cfg; if not applicable, then try config XML.
-    Properties zkProperties = makeZKPropsFromZooCfg(conf);
-
-    if (zkProperties != null) {
-      return getZKQuorumServersString(zkProperties);
-    }
-
     return getZKQuorumServersStringFromHbaseConfig(conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index fdd9e92..2fe8fd7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -158,9 +158,6 @@ public final class HConstants {
   /** Name of ZooKeeper quorum configuration parameter. */
   public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
 
-  /** Name of ZooKeeper config file in conf/ directory. */
-  public static final String ZOOKEEPER_CONFIG_NAME = "zoo.cfg";
-
   /** Common prefix of ZooKeeper configuration properties */
   public static final String ZK_CFG_PROPERTY_PREFIX =
       "hbase.zookeeper.property.";
@@ -977,10 +974,6 @@ public final class HConstants {
   public static final Cell NO_NEXT_INDEXED_KEY = new KeyValue();
   /** delimiter used between portions of a region name */
   public static final int DELIMITER = ',';
-  public static final String HBASE_CONFIG_READ_ZOOKEEPER_CONFIG =
-      "hbase.config.read.zookeeper.config";
-  public static final boolean DEFAULT_HBASE_CONFIG_READ_ZOOKEEPER_CONFIG =
-      false;
 
   /**
    * QOS attributes: these attributes are used to demarcate RPC call processing

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index fbaff1c..49dd9eb 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -432,15 +432,7 @@ possible configurations would overwhelm and obscure the important.
     and will not be downgraded.  ZooKeeper versions before 3.4 do not support multi-update and
     will not fail gracefully if multi-update is invoked (see ZOOKEEPER-1495).</description>
   </property>
-  <property>
-    <name>hbase.config.read.zookeeper.config</name>
-    <value>false</value>
-    <description>
-        Set to true to allow HBaseConfiguration to read the
-        zoo.cfg file for ZooKeeper properties. Switching this to true
-        is not recommended, since the functionality of reading ZK
-        properties from a zoo.cfg file has been deprecated.</description>
-  </property>
+
   <!--
   Beginning of properties that are directly mapped from ZooKeeper's zoo.cfg.
   All properties with an "hbase.zookeeper.property." prefix are converted for

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-rest/src/test/resources/zoo.cfg
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/resources/zoo.cfg b/hbase-rest/src/test/resources/zoo.cfg
deleted file mode 100644
index a7b8ec8..0000000
--- a/hbase-rest/src/test/resources/zoo.cfg
+++ /dev/null
@@ -1,43 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-# The number of milliseconds of each tick
-tickTime=2000
-# The number of ticks that the initial
-# synchronization phase can take
-initLimit=10
-# The number of ticks that can pass between
-# sending a request and getting an acknowledgement
-syncLimit=5
-# the directory where the snapshot is stored.
-# do not use /tmp for storage, /tmp here is just
-# example sakes.
-dataDir=/tmp/hbase-test-zookeeper-deleteme
-# the port at which the clients will connect
-clientPort=9999
-#
-# Be sure to read the maintenance section of the
-# administrator guide before turning on autopurge.
-#
-# http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#sc_maintenance
-#
-# The number of snapshots to retain in dataDir
-autopurge.snapRetainCount=3
-# Purge task interval in hours
-# Set to "0" to disable auto purge feature
-autopurge.purgeInterval=1
-
-server.1=i-am-a-test-server:7999:8999

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 244a6fe..229d11d 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -91,7 +91,6 @@
             <exclude>log4j.properties</exclude>
             <exclude>mapred-queues.xml</exclude>
             <exclude>mapred-site.xml</exclude>
-            <exclude>zoo.cfg</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
index 73483da..9f96c9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
@@ -20,12 +20,12 @@
 package org.apache.hadoop.hbase.zookeeper;
 
 import java.util.Properties;
-import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Tool for reading ZooKeeper servers from HBase XML configuration and producing
@@ -39,18 +39,15 @@ public class ZKServerTool {
    */
   public static void main(String args[]) {
     Configuration conf = HBaseConfiguration.create();
-    // Note that we do not simply grab the property
-    // HConstants.ZOOKEEPER_QUORUM from the HBaseConfiguration because the
-    // user may be using a zoo.cfg file.
     Properties zkProps = ZKConfig.makeZKProps(conf);
-    for (Entry<Object, Object> entry : zkProps.entrySet()) {
-      String key = entry.getKey().toString().trim();
-      String value = entry.getValue().toString().trim();
-      if (key.startsWith("server.")) {
-        String[] parts = value.split(":");
-        String host = parts[0];
-        System.out.println("ZK host:" + host);
-      }
+    String quorum = zkProps.getProperty(HConstants.ZOOKEEPER_QUORUM);
+
+    String[] values = quorum.split(",");
+    for (String value : values) {
+      String[] parts = value.split(":");
+      String host = parts[0];
+      System.out.println("ZK host:" + host);
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
index a8a61ad..1a91998 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
@@ -18,26 +18,23 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
-import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static junit.framework.Assert.assertEquals;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * Test for HQuorumPeer.
@@ -88,51 +85,6 @@ public class TestHQuorumPeer {
     conf.set(HConstants.ZOOKEEPER_QUORUM, oldValue);
   }
 
-  @Test public void testConfigInjection() throws Exception {
-    String s =
-      "dataDir=" + this.dataDir.toString() + "\n" +
-      "clientPort=2181\n" +
-      "initLimit=2\n" +
-      "syncLimit=2\n" +
-      "server.0=${hbase.master.hostname}:2888:3888\n" +
-      "server.1=server1:2888:3888\n" +
-      "server.2=server2:2888:3888\n";
-
-    System.setProperty("hbase.master.hostname", "localhost");
-    InputStream is = new ByteArrayInputStream(s.getBytes());
-    Configuration conf = TEST_UTIL.getConfiguration();
-    Properties properties = ZKConfig.parseZooCfg(conf, is);
-
-    assertEquals(this.dataDir.toString(), properties.get("dataDir"));
-    assertEquals(Integer.valueOf(2181),
-      Integer.valueOf(properties.getProperty("clientPort")));
-    assertEquals("localhost:2888:3888", properties.get("server.0"));
-
-    HQuorumPeer.writeMyID(properties);
-    QuorumPeerConfig config = new QuorumPeerConfig();
-    config.parseProperties(properties);
-
-    assertEquals(this.dataDir.toString(), config.getDataDir());
-    assertEquals(2181, config.getClientPortAddress().getPort());
-    Map<Long,QuorumServer> servers = config.getServers();
-    assertEquals(3, servers.size());
-    assertTrue(servers.containsKey(Long.valueOf(0)));
-    QuorumServer server = servers.get(Long.valueOf(0));
-    assertEquals("localhost", server.addr.getHostName());
-
-    // Override with system property.
-    System.setProperty("hbase.master.hostname", "foo.bar");
-    is = new ByteArrayInputStream(s.getBytes());
-    properties = ZKConfig.parseZooCfg(conf, is);
-    assertEquals("foo.bar:2888:3888", properties.get("server.0"));
-
-    config.parseProperties(properties);
-
-    servers = config.getServers();
-    server = servers.get(Long.valueOf(0));
-    assertEquals("foo.bar", server.addr.getHostName());
-  }
-
   @Test public void testShouldAssignDefaultZookeeperClientPort() {
     Configuration config = HBaseConfiguration.create();
     config.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
index eae7c2a..8f5961f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -19,39 +19,27 @@ package org.apache.hadoop.hbase.zookeeper;
 
 import java.util.Properties;
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({MiscTests.class, SmallTests.class})
 public class TestZKConfig {
+
   @Test
   public void testZKConfigLoading() throws Exception {
-    // Test depends on test resource 'zoo.cfg' at src/test/resources/zoo.cfg
     Configuration conf = HBaseConfiguration.create();
-    // Test that by default we do not pick up any property from the zoo.cfg
-    // since that feature is to be deprecated and removed. So we should read only
-    // from the config instance (i.e. via hbase-default.xml and hbase-site.xml)
+    // Test that we read only from the config instance
+    // (i.e. via hbase-default.xml and hbase-site.xml)
     conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181);
     Properties props = ZKConfig.makeZKProps(conf);
-    Assert.assertEquals(
-        "Property client port should have been default from the HBase config",
-        "2181",
-        props.getProperty("clientPort"));
-    // Test deprecated zoo.cfg read support by explicitly enabling it and
-    // thereby relying on our test resource zoo.cfg to be read.
-    // We may remove this test after a higher release (i.e. post-deprecation).
-    conf.setBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, true);
-    props = ZKConfig.makeZKProps(conf);
-    Assert.assertEquals(
-        "Property client port should have been from zoo.cfg",
-        "9999",
-        props.getProperty("clientPort"));
+    Assert.assertEquals("Property client port should have been default from the HBase config",
+                        "2181",
+                        props.getProperty("clientPort"));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-server/src/test/resources/zoo.cfg
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/zoo.cfg b/hbase-server/src/test/resources/zoo.cfg
deleted file mode 100644
index a7b8ec8..0000000
--- a/hbase-server/src/test/resources/zoo.cfg
+++ /dev/null
@@ -1,43 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-# The number of milliseconds of each tick
-tickTime=2000
-# The number of ticks that the initial
-# synchronization phase can take
-initLimit=10
-# The number of ticks that can pass between
-# sending a request and getting an acknowledgement
-syncLimit=5
-# the directory where the snapshot is stored.
-# do not use /tmp for storage, /tmp here is just
-# example sakes.
-dataDir=/tmp/hbase-test-zookeeper-deleteme
-# the port at which the clients will connect
-clientPort=9999
-#
-# Be sure to read the maintenance section of the
-# administrator guide before turning on autopurge.
-#
-# http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#sc_maintenance
-#
-# The number of snapshots to retain in dataDir
-autopurge.snapRetainCount=3
-# Purge task interval in hours
-# Set to "0" to disable auto purge feature
-autopurge.purgeInterval=1
-
-server.1=i-am-a-test-server:7999:8999

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/hbase-shell/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml
index f4d54f5..8c01992 100644
--- a/hbase-shell/pom.xml
+++ b/hbase-shell/pom.xml
@@ -88,7 +88,6 @@
             <exclude>log4j.properties</exclude>
             <exclude>mapred-queues.xml</exclude>
             <exclude>mapred-site.xml</exclude>
-            <exclude>zoo.cfg</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 814502b..0e95fb0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -644,7 +644,6 @@
               <exclude>log4j.properties</exclude>
               <exclude>mapred-queues.xml</exclude>
               <exclude>mapred-site.xml</exclude>
-              <exclude>zoo.cfg</exclude>
             </excludes>
           </configuration>
         </plugin>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/src/main/asciidoc/_chapters/hbase-default.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc
index bf56dd3..8df9b17 100644
--- a/src/main/asciidoc/_chapters/hbase-default.adoc
+++ b/src/main/asciidoc/_chapters/hbase-default.adoc
@@ -605,21 +605,7 @@ Instructs HBase to make use of ZooKeeper's multi-update functionality.
 .Default
 `true`
 
-  
-[[hbase.config.read.zookeeper.config]]
-*`hbase.config.read.zookeeper.config`*::
-+
-.Description
-
-        Set to true to allow HBaseConfiguration to read the
-        zoo.cfg file for ZooKeeper properties. Switching this to true
-        is not recommended, since the functionality of reading ZK
-        properties from a zoo.cfg file has been deprecated.
-+
-.Default
-`false`
 
-  
 [[hbase.zookeeper.property.initLimit]]
 *`hbase.zookeeper.property.initLimit`*::
 +
@@ -2251,4 +2237,3 @@ The percent of region server RPC threads failed to abort RS.
 .Default
 `0.5`
 
-  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1fcee869/src/main/asciidoc/_chapters/zookeeper.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/zookeeper.adoc b/src/main/asciidoc/_chapters/zookeeper.adoc
index f6134b7..3266964 100644
--- a/src/main/asciidoc/_chapters/zookeeper.adoc
+++ b/src/main/asciidoc/_chapters/zookeeper.adoc
@@ -35,7 +35,7 @@ You can also manage the ZooKeeper ensemble independent of HBase and just point H
 To toggle HBase management of ZooKeeper, use the `HBASE_MANAGES_ZK` variable in _conf/hbase-env.sh_.
 This variable, which defaults to `true`, tells HBase whether to start/stop the ZooKeeper ensemble servers as part of HBase start/stop.
 
-When HBase manages the ZooKeeper ensemble, you can specify ZooKeeper configuration using its native _zoo.cfg_ file, or, the easier option is to just specify ZooKeeper options directly in _conf/hbase-site.xml_.
+When HBase manages the ZooKeeper ensemble, you can specify ZooKeeper configuration directly in _conf/hbase-site.xml_.
 A ZooKeeper configuration option can be set as a property in the HBase _hbase-site.xml_ XML configuration file by prefacing the ZooKeeper option name with `hbase.zookeeper.property`.
 For example, the `clientPort` setting in ZooKeeper can be changed by setting the `hbase.zookeeper.property.clientPort` property.
 For all default values used by HBase, including ZooKeeper configuration, see <<hbase_default_configurations,hbase default configurations>>.
@@ -124,8 +124,7 @@ To point HBase at an existing ZooKeeper cluster, one that is not managed by HBas
   export HBASE_MANAGES_ZK=false
 ----
 
-Next set ensemble locations and client port, if non-standard, in _hbase-site.xml_, or add a suitably configured _zoo.cfg_ to HBase's _CLASSPATH_.
-HBase will prefer the configuration found in _zoo.cfg_ over any settings in _hbase-site.xml_.
+Next set ensemble locations and client port, if non-standard, in _hbase-site.xml_.
 
 When HBase manages ZooKeeper, it will start/stop the ZooKeeper servers as a part of the regular start/stop scripts.
 If you would like to run ZooKeeper yourself, independent of HBase start/stop, you would do the following
@@ -312,21 +311,23 @@ Modify your _hbase-site.xml_ on each node that will run a master or regionserver
     <name>hbase.cluster.distributed</name>
     <value>true</value>
   </property>
+  <property>
+    <name>hbase.zookeeper.property.authProvider.1</name>
+    <value>org.apache.zookeeper.server.auth.SASLAuthenticationProvider</value>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.kerberos.removeHostFromPrincipal</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.kerberos.removeRealmFromPrincipal</name>
+    <value>true</value>
+  </property>
 </configuration>
 ----
 
 where `$ZK_NODES` is the comma-separated list of hostnames of the Zookeeper Quorum hosts.
 
-Add a _zoo.cfg_ for each Zookeeper Quorum host containing:
-
-[source,java]
-----
-
-authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
-kerberos.removeHostFromPrincipal=true
-kerberos.removeRealmFromPrincipal=true
-----
-
 Also on each of these hosts, create a JAAS configuration file containing:
 
 [source,java]