You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2019/09/13 01:40:59 UTC

[impala] 02/02: IMPALA-8586: Support download URLs for CDP

This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit da0ab1d41ae777fbd7094d44628dfee1ff0fc8fe
Author: Joe McDonnell <jo...@cloudera.com>
AuthorDate: Mon May 20 15:36:25 2019 -0700

    IMPALA-8586: Support download URLs for CDP
    
    bin/bootstrap_toolchain.py has accumulated complexity over time.
    CDH, CDP, and the native toolchain all use different download
    machinery and naming. One feature that is needed on the CDP side
    is the ability to specify the download URL in an IMPALA_*_URL
    environment variable.
    
    This adds that support and refactors CDH and native toolchain
    downloads to use the new system. This is essentially a rewrite
    of bin/bootstrap_toolchain.py.
    
    Currently, there are multiple phases of downloads, each with their
    own download functions and peculiarities to account for package
    names and destinations for downloads. This changes the logic
    so that a package will generate a DownloadUnpackTarball that is
    completely resolved. It contains everything about what to download
    and where to put it as well as a needs_download() function and a
    download() function. Once there is a list of DownloadUnpackTarball
    objects, they can all be downloaded and unpacked in a single phase.
    This implements different types of packages as subclasses of
    DownloadUnpackTarball. Since most subclasses want to be able to
    construct URLs and archive names using templates, the
    TemplatedDownloadUnpackTarball takes the same arguments as
    DownloadUnpackTarball along with a map of template substitutions,
    which are applied to all string arguments.
    
    Kudu requires special handling and gets its own set of subclasses
    to handle various subtleties like toolchain vs CDH Kudu, the Kudu
    stub, and making sure that the "kudu" package and the "kudu-java"
    package don't confuse each other.
    
    As part of this change, USE_CDP_HIVE=true now uses the CDP version
    of HBase rather than always using the CDH version.
    
    Change-Id: I67824fd82b820e68e9f5c87939ec94ca6abadb8c
    Reviewed-on: http://gerrit.cloudera.org:8080/13432
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 bin/bootstrap_toolchain.py | 823 ++++++++++++++++++++++++++-------------------
 bin/impala-config.sh       |  63 +++-
 impala-parent/pom.xml      |   4 +-
 3 files changed, 523 insertions(+), 367 deletions(-)

diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index 6be838f..1abdb08 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -16,46 +16,68 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-# The purpose of this script is to download prebuilt binaries and jar files to satisfy the
-# third-party dependencies for Impala. The script checks for the presence of IMPALA_HOME
-# and IMPALA_TOOLCHAIN. IMPALA_HOME indicates that the environment is correctly setup and
-# that we can deduce the version settings of the dependencies from the environment.
-# IMPALA_TOOLCHAIN indicates the location where the prebuilt artifacts should be extracted
-# to. If DOWNLOAD_CDH_COMPONENTS is set to true, this script will also download and
-# extract the CDH components (i.e. Hadoop, Hive, HBase and Sentry) into
-# CDH_COMPONENTS_HOME.
+# The purpose of this script is to download prebuilt binaries and jar files to satisfy
+# the third-party dependencies for Impala. The script expects bin/impala-config.sh to be
+# sourced to initialize various environment variables (including environment variables
+# specifying the versions for components). It verifies that bin/impala-config.sh
+# has been sourced by verifying that IMPALA_HOME is set. This script will fail if an
+# expected environment variable is not present.
 #
-# Kudu can be downloaded either from the toolchain or as a CDH component, depending on the
-# value of USE_CDH_KUDU. If KUDU_IS_SUPPORTED is false, we download the toolchain Kudu and
-# use the symbols to compile a non-functional stub library so that Impala has something to
-# link against.
+# The following environment variables control the behavior of this script:
+# IMPALA_TOOLCHAIN - Directory in which to place the artifacts. It can be useful to
+#   override this to share a toolchain directory between multiple checkouts of Impala
+#   or to use a cached copy to avoid downloading a new one.
+# IMPALA_TOOLCHAIN_HOST - The host to use for downloading the artifacts
+# CDH_COMPONENTS_HOME - Directory to store CDH Hadoop component artifacts
+# CDP_COMPONENTS_HOME - Directory to store CDP Hadoop component artifacts
+# CDH_BUILD_NUMBER - CDH Hadoop components are built with consistent versions so that
+#   Hadoop, Hive, Kudu, etc are all built with versions that are compatible with each
+#   other. The way to specify a single consistent set of components is via a build
+#   number. This determines the location in s3 to get the artifacts.
+# CDP_BUILD_NUMBER - The CDP equivalent of a CDH_BUILD_NUMBER.
+# USE_CDP_HIVE - If false, this will use the CDH version of all Hadoop components
+#   (except Ranger, which is CDP only). If true, this will use the CDP version of all
+#   Hadoop components (except Sentry, which is CDH only).
+# DOWNLOAD_CDH_COMPONENTS - When set to true, this script will also download and extract
+#   the CDH/CDP Hadoop components (i.e. Hadoop, Hive, HBase, Sentry, Ranger, etc) into
+#   CDH_COMPONENTS_HOME/CDP_COMPONENTS_HOME as appropriate.
+# USE_CDH_KUDU - Kudu can be downloaded either from the toolchain or as a CDH component,
+#   depending on the value of USE_CDH_KUDU.
+# KUDU_IS_SUPPORTED - If KUDU_IS_SUPPORTED is false, Kudu is disabled and we download
+#   the toolchain Kudu and use the symbols to compile a non-functional stub library so
+#   that Impala has something to link against.
+# IMPALA_<PACKAGE>_VERSION - The version expected for <PACKAGE>. This is typically
+#   configured in bin/impala-config.sh and must exist for every package. This is used
+#   to construct an appropriate URL and expected archive name.
+# IMPALA_<PACKAGE>_URL - This overrides the download URL for <PACKAGE>. The URL must
+#   end with the expected archive name for this package. This is usually used in
+#   bin/impala-config-branch.sh or bin/impala-config-local.sh when using custom
+#   versions or packages. When this is not specified, packages are downloaded from
+#   an S3 bucket named native-toolchain, and the exact URL is based on
+#   IMPALA_<PACKAGE>_VERSION as well as the OS version being built on.
 #
-# By default, packages are downloaded from an S3 bucket named native-toolchain.
-# The exact URL is based on IMPALA_<PACKAGE>_VERSION environment variables
-# (configured in impala-config.sh) as well as the OS version being built on.
-# The URL can be overridden with an IMPALA_<PACKAGE>_URL environment variable
-# set in impala-config-{local,branch}.sh.
-#
-# The script is called as follows without any additional parameters:
-#
-#     python bootstrap_toolchain.py
+# The script is directly executable, and it takes no parameters:
+#     ./bootstrap_toolchain.py
+# It should NOT be run via 'python bootstrap_toolchain.py', as it relies on a specific
+# python environment.
 import logging
 import glob
 import multiprocessing.pool
 import os
 import random
 import re
+# TODO: This file should be runnable without using impala-python, and system python
+# does not have 'sh' available. Rework code to avoid importing sh (and anything else
+# that gets in the way).
 import sh
 import shutil
 import subprocess
 import sys
 import tempfile
 import time
-import traceback
 
 from collections import namedtuple
-
-TOOLCHAIN_HOST = "https://native-toolchain.s3.amazonaws.com/build"
+from string import Template
 
 # Maps return values from 'lsb_release -irs' to the corresponding OS labels for both the
 # toolchain and the CDH components.
@@ -82,47 +104,283 @@ OS_MAPPING = [
   OsMapping('ubuntu18.04', "ec2-package-ubuntu-18-04", "ubuntu1804")
 ]
 
-class Package(object):
+
+def wget_and_unpack_package(download_path, file_name, destination, wget_no_clobber):
+  if not download_path.endswith("/" + file_name):
+    raise Exception("URL {0} does not match with expected file_name {1}"
+        .format(download_path, file_name))
+  NUM_ATTEMPTS = 3
+  for attempt in range(1, NUM_ATTEMPTS + 1):
+    logging.info("Downloading {0} to {1}/{2} (attempt {3})".format(
+      download_path, destination, file_name, attempt))
+    # --no-clobber avoids downloading the file if a file with the name already exists
+    try:
+      sh.wget(download_path, directory_prefix=destination, no_clobber=wget_no_clobber)
+      break
+    except Exception, e:
+      if attempt == NUM_ATTEMPTS:
+        raise
+      logging.error("Download failed; retrying after sleep: " + str(e))
+      time.sleep(10 + random.random() * 5)  # Sleep between 10 and 15 seconds.
+  logging.info("Extracting {0}".format(file_name))
+  sh.tar(z=True, x=True, f=os.path.join(destination, file_name), directory=destination)
+  sh.rm(os.path.join(destination, file_name))
+
+
+class DownloadUnpackTarball(object):
   """
-  Represents a package to be downloaded. A version, if not specified
-  explicitly, is retrieved from the environment variable IMPALA_<NAME>_VERSION.
-  URLs are retrieved from IMPALA_<NAME>_URL, but are optional.
+  The basic unit of work for bootstrapping the toolchain is:
+   - check if a package is already present (via the needs_download() method)
+   - if it is not, download a tarball and unpack it into the appropriate directory
+     (via the download() method)
+  In this base case, everything is known: the url to download from, the archive to
+  unpack, and the destination directory.
   """
-  def __init__(self, name, version=None, url=None):
-    self.name = name
-    self.version = version
+  def __init__(self, url, archive_name, destination_basedir, directory_name, makedir):
     self.url = url
-    package_env_name = name.replace("-", "_").upper()
-    if self.version is None:
-      version_env_var = "IMPALA_{0}_VERSION".format(package_env_name)
+    self.archive_name = archive_name
+    assert self.archive_name.endswith(".tar.gz")
+    self.archive_basename = self.archive_name.replace(".tar.gz", "")
+    self.destination_basedir = destination_basedir
+    # destination base directory must exist
+    assert os.path.isdir(self.destination_basedir)
+    self.directory_name = directory_name
+    self.makedir = makedir
+
+  def pkg_directory(self):
+    return os.path.join(self.destination_basedir, self.directory_name)
+
+  def needs_download(self):
+    if os.path.isdir(self.pkg_directory()): return False
+    return True
 
-      self.version = os.environ.get(version_env_var)
-      if not self.version:
+  def download(self):
+    unpack_dir = self.pkg_directory()
+    if self.makedir:
+      # Download and unpack in a temp directory, which we'll later move into place
+      download_dir = tempfile.mkdtemp(dir=self.destination_basedir)
+    else:
+      download_dir = self.destination_basedir
+    try:
+      wget_and_unpack_package(self.url, self.archive_name, download_dir, False)
+    except:  # noqa
+      # Clean up any partially-unpacked result.
+      if os.path.isdir(unpack_dir):
+        shutil.rmtree(unpack_dir)
+      if os.path.isdir(download_dir):
+        shutil.rmtree(download_dir)
+      raise
+    if self.makedir:
+      os.rename(download_dir, unpack_dir)
+
+
+class TemplatedDownloadUnpackTarball(DownloadUnpackTarball):
+  def __init__(self, url_tmpl, archive_name_tmpl, destination_basedir_tmpl,
+               directory_name_tmpl, makedir, template_subs):
+    url = self.__do_substitution(url_tmpl, template_subs)
+    archive_name = self.__do_substitution(archive_name_tmpl, template_subs)
+    destination_basedir = self.__do_substitution(destination_basedir_tmpl, template_subs)
+    directory_name = self.__do_substitution(directory_name_tmpl, template_subs)
+    super(TemplatedDownloadUnpackTarball, self).__init__(url, archive_name,
+        destination_basedir, directory_name, makedir)
+
+  def __do_substitution(self, template, template_subs):
+    return Template(template).substitute(**template_subs)
+
+
+class EnvVersionedPackage(TemplatedDownloadUnpackTarball):
+  def __init__(self, name, url_prefix_tmpl, destination_basedir, explicit_version=None,
+               archive_basename_tmpl=None, unpack_directory_tmpl=None, makedir=False,
+               template_subs_in={}):
+    template_subs = template_subs_in
+    template_subs["name"] = name
+    template_subs["version"] = self.__compute_version(name, explicit_version)
+    # The common case is that X.tar.gz unpacks to X directory. archive_basename_tmpl
+    # allows overriding the value of X (which defaults to ${name}-${version}).
+    # If X.tar.gz unpacks to Y directory, then unpack_directory_tmpl allows overriding Y.
+    if archive_basename_tmpl is None:
+      archive_basename_tmpl = "${name}-${version}"
+    archive_name_tmpl = archive_basename_tmpl + ".tar.gz"
+    if unpack_directory_tmpl is None:
+      unpack_directory_tmpl = archive_basename_tmpl
+    url_tmpl = self.__compute_url(name, archive_name_tmpl, url_prefix_tmpl)
+    super(EnvVersionedPackage, self).__init__(url_tmpl, archive_name_tmpl,
+        destination_basedir, unpack_directory_tmpl, makedir, template_subs)
+
+  def __compute_version(self, name, explicit_version):
+    if explicit_version is not None:
+      return explicit_version
+    else:
+      # When getting the version from the environment, we need to standardize the name
+      # to match expected environment variables.
+      std_env_name = name.replace("-", "_").upper()
+      version_env_var = "IMPALA_{0}_VERSION".format(std_env_name)
+      env_version = os.environ.get(version_env_var)
+      if not env_version:
         raise Exception("Could not find version for {0} in environment var {1}".format(
           name, version_env_var))
-    if self.url is None:
-      url_env_var = "IMPALA_{0}_URL".format(package_env_name)
-      self.url = os.environ.get(url_env_var)
-
-
-class CdpComponent(object):
-  def __init__(self, basename, makedir=False, pkg_directory=None):
-    """
-    basename: the name of the file to be downloaded, without its .tar.gz suffix
-    makedir: if false, it is assumed that the downloaded tarball will expand
-             into a directory with the same name as 'basename'. If True, we
-             assume that the tarball doesn't have any top-level directory,
-             and so we need to manually create a directory within which to
-             expand the tarball.
-    """
-    self.basename = basename
-    self.makedir = makedir
-    cdp_components_home = os.environ.get("CDP_COMPONENTS_HOME")
-    if cdp_components_home is None:
-      raise Exception("CDP_COMPONENTS_HOME is not set. Cannot determine the "
-                      "component package directory")
-    self.pkg_directory = "{0}/{1}".format(cdp_components_home,
-                            pkg_directory if pkg_directory else basename)
+      return env_version
+
+  def __compute_url(self, name, archive_name_tmpl, url_prefix_tmpl):
+    # The URL defined in the environment (IMPALA_*_URL) takes precedence. If that is
+    # not defined, use the standard URL (url_prefix + archive_name)
+    std_env_name = name.replace("-", "_").upper()
+    url_env_var = "IMPALA_{0}_URL".format(std_env_name)
+    url_tmpl = os.environ.get(url_env_var)
+    if not url_tmpl:
+      url_tmpl = os.path.join(url_prefix_tmpl, archive_name_tmpl)
+    return url_tmpl
+
+
+class ToolchainPackage(EnvVersionedPackage):
+  def __init__(self, name, explicit_version=None, platform_release=None):
+    toolchain_root = os.environ.get("IMPALA_TOOLCHAIN")
+    if not toolchain_root:
+      logging.error("Impala environment not set up correctly, make sure "
+          "$IMPALA_TOOLCHAIN is set.")
+      sys.exit(1)
+    compiler = "gcc-{0}".format(os.environ["IMPALA_GCC_VERSION"])
+    label = get_platform_release_label(release=platform_release).toolchain
+    toolchain_build_id = os.environ["IMPALA_TOOLCHAIN_BUILD_ID"]
+    toolchain_host = os.environ["IMPALA_TOOLCHAIN_HOST"]
+    template_subs = {'compiler': compiler, 'label': label,
+                     'toolchain_build_id': toolchain_build_id,
+                     'toolchain_host': toolchain_host}
+    archive_basename_tmpl = "${name}-${version}-${compiler}-${label}"
+    url_prefix_tmpl = "https://${toolchain_host}/build/${toolchain_build_id}/" + \
+        "${name}/${version}-${compiler}/"
+    unpack_directory_tmpl = "${name}-${version}"
+    super(ToolchainPackage, self).__init__(name, url_prefix_tmpl, toolchain_root,
+                                           explicit_version=explicit_version,
+                                           archive_basename_tmpl=archive_basename_tmpl,
+                                           unpack_directory_tmpl=unpack_directory_tmpl,
+                                           template_subs_in=template_subs)
+
+  def needs_download(self):
+    # If the directory doesn't exist, we need the download
+    unpack_dir = self.pkg_directory()
+    if not os.path.isdir(unpack_dir): return True
+    version_file = os.path.join(unpack_dir, "toolchain_package_version.txt")
+    if not os.path.exists(version_file): return True
+    with open(version_file, "r") as f:
+      return f.read().strip() != self.archive_basename
+
+  def download(self):
+    # Remove the existing package directory if it exists (since this has additional
+    # conditions as part of needs_download())
+    unpack_dir = self.pkg_directory()
+    if os.path.exists(unpack_dir):
+      logging.info("Removing existing package directory {0}".format(unpack_dir))
+      shutil.rmtree(unpack_dir)
+    super(ToolchainPackage, self).download()
+    # Write the toolchain_package_version.txt file
+    version_file = os.path.join(unpack_dir, "toolchain_package_version.txt")
+    with open(version_file, "w") as f:
+      f.write(self.archive_basename)
+
+
+class CdhComponent(EnvVersionedPackage):
+  def __init__(self, name, explicit_version=None, archive_basename_tmpl=None,
+               unpack_directory_tmpl=None):
+    # Compute the CDH base URL (based on the IMPALA_TOOLCHAIN_HOST and CDH_BUILD_NUMBER)
+    if "IMPALA_TOOLCHAIN_HOST" not in os.environ or "CDH_BUILD_NUMBER" not in os.environ:
+      logging.error("Impala environment not set up correctly, make sure "
+                    "impala-config.sh is sourced.")
+      sys.exit(1)
+    template_subs = {"toolchain_host": os.environ["IMPALA_TOOLCHAIN_HOST"],
+                     "cdh_build_number": os.environ["CDH_BUILD_NUMBER"]}
+    url_prefix_tmpl = "https://${toolchain_host}/build/cdh_components/" + \
+        "${cdh_build_number}/tarballs/"
+
+    # Get the output base directory from CDH_COMPONENTS_HOME
+    destination_basedir = os.environ["CDH_COMPONENTS_HOME"]
+    super(CdhComponent, self).__init__(name, url_prefix_tmpl, destination_basedir,
+                                       explicit_version=explicit_version,
+                                       archive_basename_tmpl=archive_basename_tmpl,
+                                       unpack_directory_tmpl=unpack_directory_tmpl,
+                                       template_subs_in=template_subs)
+
+
+class CdpComponent(EnvVersionedPackage):
+  def __init__(self, name, explicit_version=None, archive_basename_tmpl=None,
+               unpack_directory_tmpl=None, makedir=False):
+    # Compute the CDH base URL (based on the IMPALA_TOOLCHAIN_HOST and CDP_BUILD_NUMBER)
+    if "IMPALA_TOOLCHAIN_HOST" not in os.environ or "CDP_BUILD_NUMBER" not in os.environ:
+      logging.error("Impala environment not set up correctly, make sure "
+                    "impala-config.sh is sourced.")
+      sys.exit(1)
+    template_subs = {"toolchain_host": os.environ["IMPALA_TOOLCHAIN_HOST"],
+                     "cdp_build_number": os.environ["CDP_BUILD_NUMBER"]}
+    url_prefix_tmpl = "https://${toolchain_host}/build/cdp_components/" + \
+        "${cdp_build_number}/tarballs/"
+
+    # Get the output base directory from CDP_COMPONENTS_HOME
+    destination_basedir = os.environ["CDP_COMPONENTS_HOME"]
+    super(CdpComponent, self).__init__(name, url_prefix_tmpl, destination_basedir,
+                                       explicit_version=explicit_version,
+                                       archive_basename_tmpl=archive_basename_tmpl,
+                                       unpack_directory_tmpl=unpack_directory_tmpl,
+                                       makedir=makedir, template_subs_in=template_subs)
+
+
+class ToolchainKudu(ToolchainPackage):
+  def __init__(self, platform_label=None):
+    super(ToolchainKudu, self).__init__('kudu', platform_release=platform_label)
+
+  def needs_download(self):
+    # This verifies that the unpack directory exists
+    if super(ToolchainKudu, self).needs_download():
+      return True
+    # Additional check to distinguish this from the Kudu Java package
+    # Regardless of the actual build type, the 'kudu' tarball will always contain a
+    # 'debug' and a 'release' directory.
+    if not os.path.exists(os.path.join(self.pkg_directory(), "debug")):
+      return True
+    # Both the pkg_directory and the debug directory exist
+    return False
+
+
+class CdhKudu(CdhComponent):
+  def __init__(self, platform_label):
+    kudu_archive_tmpl = "kudu-${version}-" + platform_label
+    # IMPALA_KUDU_URL can contain '%(platform_label)', which needs to be replaced
+    # with the platform. We override this in os.environ so that it is picked up
+    # in EnvVersionedPackage.
+    kudu_url = os.environ.get("IMPALA_KUDU_URL")
+    if kudu_url:
+       kudu_url = kudu_url.replace("%(platform_label)", platform_label)
+       os.environ["IMPALA_KUDU_URL"] = kudu_url
+    super(CdhKudu, self).__init__('kudu',
+                                  archive_basename_tmpl=kudu_archive_tmpl,
+                                  unpack_directory_tmpl="kudu-${version}")
+
+  def needs_download(self):
+    # This verifies that the unpack directory exists
+    if super(CdhKudu, self).needs_download():
+      return True
+    # Additional check to distinguish this from the Kudu Java package
+    # Regardless of the actual build type, the 'kudu' tarball will always contain a
+    # 'debug' and a 'release' directory.
+    if not os.path.exists(os.path.join(self.pkg_directory(), "debug")):
+      return True
+    # Both the pkg_directory and the debug directory exist
+    return False
+
+
+class CdhKuduJava(CdhComponent):
+  def __init__(self):
+    super(CdhKuduJava, self).__init__('kudu-java',
+                                      archive_basename_tmpl="kudu-${version}")
+
+  def needs_download(self):
+    # This verify that the unpack directory exists
+    if super(CdhKuduJava, self).needs_download():
+      return True
+    # Additional check to distinguish this from the Kudu package
+    # There should be jars under the kudu directory.
+    if len(glob.glob("{0}/*jar".format(self.pkg_directory()))) == 0:
+      return True
+    return False
 
 
 def try_get_platform_release_label():
@@ -162,67 +420,6 @@ def get_platform_release_label(release=None):
 
   raise Exception("Could not find package label for OS version: {0}.".format(release))
 
-def wget_and_unpack_package(download_path, file_name, destination, wget_no_clobber):
-  if not download_path.endswith("/" + file_name):
-    raise Exception("URL {0} does not match with expected file_name {1}"
-        .format(download_path, file_name))
-  NUM_ATTEMPTS = 3
-  for attempt in range(1, NUM_ATTEMPTS + 1):
-    logging.info("Downloading {0} to {1}/{2} (attempt {3})".format(
-      download_path, destination, file_name, attempt))
-    # --no-clobber avoids downloading the file if a file with the name already exists
-    try:
-      sh.wget(download_path, directory_prefix=destination, no_clobber=wget_no_clobber)
-      break
-    except Exception, e:
-      if attempt == NUM_ATTEMPTS:
-        raise
-      logging.error("Download failed; retrying after sleep: " + str(e))
-      time.sleep(10 + random.random() * 5) # Sleep between 10 and 15 seconds.
-  logging.info("Extracting {0}".format(file_name))
-  sh.tar(z=True, x=True, f=os.path.join(destination, file_name), directory=destination)
-  sh.rm(os.path.join(destination, file_name))
-
-def download_package(destination, package, compiler, platform_release=None):
-  remove_existing_package(destination, package.name, package.version)
-
-  toolchain_build_id = os.environ["IMPALA_TOOLCHAIN_BUILD_ID"]
-  label = get_platform_release_label(release=platform_release).toolchain
-  format_params = {'product': package.name, 'version': package.version,
-      'compiler': compiler, 'label': label, 'toolchain_build_id': toolchain_build_id}
-  file_name = "{product}-{version}-{compiler}-{label}.tar.gz".format(**format_params)
-  format_params['file_name'] = file_name
-  if package.url is None:
-    url_path = "/{toolchain_build_id}/{product}/{version}-{compiler}/{file_name}".format(
-        **format_params)
-    download_path = TOOLCHAIN_HOST + url_path
-  else:
-    download_path = package.url
-
-  wget_and_unpack_package(download_path, file_name, destination, True)
-
-def bootstrap(toolchain_root, packages):
-  """Downloads and unpacks each package in the list `packages` into `toolchain_root` if it
-  doesn't exist already.
-  """
-  if not try_get_platform_release_label() \
-     or not try_get_platform_release_label().toolchain:
-    check_custom_toolchain(toolchain_root, packages)
-    return
-
-  # Detect the compiler
-  compiler = "gcc-{0}".format(os.environ["IMPALA_GCC_VERSION"])
-
-  def handle_package(p):
-    if check_for_existing_package(toolchain_root, p.name, p.version, compiler):
-      return
-    if p.name != "kudu" or os.environ["KUDU_IS_SUPPORTED"] == "true":
-      download_package(toolchain_root, p, compiler)
-    else:
-      build_kudu_stub(toolchain_root, p.version, compiler)
-    write_version_file(toolchain_root, p.name, p.version, compiler,
-        get_platform_release_label().toolchain)
-  execute_many(handle_package, packages)
 
 def check_output(cmd_args):
   """Run the command and return the output. Raise an exception if the command returns
@@ -236,20 +433,12 @@ def check_output(cmd_args):
         % (cmd_args, process.returncode, stdout))
   return stdout
 
-def package_directory(toolchain_root, pkg_name, pkg_version):
-  dir_name = "{0}-{1}".format(pkg_name, pkg_version)
-  return os.path.join(toolchain_root, dir_name)
-
-def version_file_path(toolchain_root, pkg_name, pkg_version):
-  return os.path.join(package_directory(toolchain_root, pkg_name, pkg_version),
-      "toolchain_package_version.txt")
 
 def check_custom_toolchain(toolchain_root, packages):
   missing = []
   for p in packages:
-    pkg_dir = package_directory(toolchain_root, p.name, p.version)
-    if not os.path.isdir(pkg_dir):
-      missing.append((p, pkg_dir))
+    if not os.path.isdir(p.pkg_directory()):
+      missing.append((p, p.pkg_directory()))
 
   if missing:
     msg = "The following packages are not in their expected locations.\n"
@@ -261,38 +450,16 @@ def check_custom_toolchain(toolchain_root, packages):
     logging.error(msg)
     raise Exception("Toolchain bootstrap failed: required packages were missing")
 
-def check_for_existing_package(toolchain_root, pkg_name, pkg_version, compiler):
-  """Return true if toolchain_root already contains the package with the correct
-  version and compiler.
-  """
-  version_file = version_file_path(toolchain_root, pkg_name, pkg_version)
-  if not os.path.exists(version_file):
-    return False
-
-  label = get_platform_release_label().toolchain
-  pkg_version_string = "{0}-{1}-{2}-{3}".format(pkg_name, pkg_version, compiler, label)
-  with open(version_file) as f:
-    return f.read().strip() == pkg_version_string
-
-def write_version_file(toolchain_root, pkg_name, pkg_version, compiler, label):
-  with open(version_file_path(toolchain_root, pkg_name, pkg_version), 'w') as f:
-    f.write("{0}-{1}-{2}-{3}".format(pkg_name, pkg_version, compiler, label))
 
-def remove_existing_package(toolchain_root, pkg_name, pkg_version):
-  dir_path = package_directory(toolchain_root, pkg_name, pkg_version)
-  if os.path.exists(dir_path):
-    logging.info("Removing existing package directory {0}".format(dir_path))
-    shutil.rmtree(dir_path)
+def build_kudu_stub(kudu_dir, gcc_dir):
+  """When Kudu isn't supported, the CentOS 7 Kudu package is downloaded from the
+     toolchain. This replaces the client lib with a stubbed client. The
+     'kudu_dir' specifies the location of the unpacked CentOS 7 Kudu package.
+     The 'gcc_dir' specifies the location of the unpacked GCC/G++."""
 
-def build_kudu_stub(toolchain_root, kudu_version, compiler):
-  # When Kudu isn't supported, the CentOS 7 package will be downloaded and the client
-  # lib will be replaced with a stubbed client.
-  download_package(toolchain_root, Package("kudu", kudu_version), compiler,
-      platform_release="centos7")
-
-  # Find the client lib files in the extracted dir. There may be several files with
+  print "Building kudu stub"
+  # Find the client lib files in the Kudu dir. There may be several files with
   # various extensions. Also there will be a debug version.
-  kudu_dir = package_directory(toolchain_root, "kudu", kudu_version)
   client_lib_paths = []
   for path, _, files in os.walk(kudu_dir):
     for file in files:
@@ -311,8 +478,7 @@ def build_kudu_stub(toolchain_root, kudu_version, compiler):
 
   # Use a newer version of binutils because on older systems the default binutils may
   # not be able to read the newer binary.
-  binutils_dir = package_directory(
-      toolchain_root, "binutils", os.environ["IMPALA_BINUTILS_VERSION"])
+  binutils_dir = ToolchainPackage("binutils").pkg_directory()
   nm_path = os.path.join(binutils_dir, "bin", "nm")
   objdump_path = os.path.join(binutils_dir, "bin", "objdump")
 
@@ -380,6 +546,7 @@ extern "C" void %s() {
 
     # Compile the library.
     stub_client_lib_path = os.path.join(stub_build_dir, "libkudu_client.so")
+    toolchain_root = os.environ.get("IMPALA_TOOLCHAIN")
     gpp = os.path.join(
         toolchain_root, "gcc-%s" % os.environ.get("IMPALA_GCC_VERSION"), "bin", "g++")
     subprocess.check_call([gpp, stub_client_src_file.name, "-shared", "-fPIC",
@@ -391,6 +558,7 @@ extern "C" void %s() {
   finally:
     shutil.rmtree(stub_build_dir)
 
+
 def execute_many(f, args):
   """
   Executes f(a) for a in args using a threadpool to execute in parallel.
@@ -400,126 +568,129 @@ def execute_many(f, args):
   pool = multiprocessing.pool.ThreadPool(processes=min(multiprocessing.cpu_count(), 4))
   return pool.map(f, args, 1)
 
-def download_cdh_components(toolchain_root, cdh_components, url_prefix):
-  """Downloads and unpacks the CDH components for a given URL prefix into
-  $CDH_COMPONENTS_HOME if not found."""
-  cdh_components_home = os.environ.get("CDH_COMPONENTS_HOME")
-  if not cdh_components_home:
-    logging.error("Impala environment not set up correctly, make sure "
-          "$CDH_COMPONENTS_HOME is set.")
-    sys.exit(1)
-
-  # Create the directory where CDH components live if necessary.
-  if not os.path.exists(cdh_components_home):
-    os.makedirs(cdh_components_home)
-
-  def download(component):
-    try:
-      # Download and unpack Kudu's java package
-      component_name = component.name
-      if component.name == "kudu-java":
-        component_name = "kudu"
-
-      # Check if the diretory already exists, and skip downloading it if it does. Since
-      # the kudu and kudu-java tarballs unpack to the same directory, we check for files
-      # in that directory expected for each package. TODO: if we change how the Kudu
-      # tarballs are packaged we can remove this special case.
-      pkg_directory = package_directory(cdh_components_home, component_name,
-          component.version)
-      if component.name == "kudu-java":
-        if len(glob.glob("%s/*jar" % pkg_directory)) > 0:
-          return
-      elif component.name == "kudu":
-        # Regardless of the actual build type, the 'kudu' tarball will always contain a
-        # 'debug' and a 'release' directory.
-        if os.path.exists(os.path.join(pkg_directory, "debug")):
-          return
-      else:
-        if os.path.isdir(pkg_directory):
-          return
-
-      platform_label = ""
-      # Kudu is the only component that's platform dependent.
-      if component.name == "kudu":
-        platform_label = "-%s" % get_platform_release_label().cdh
-      # Download the package if it doesn't exist
-      file_name = "{0}-{1}{2}.tar.gz".format(
-          component_name, component.version, platform_label)
-
-      if component.url is None:
-        download_path = url_prefix + file_name
-      else:
-        download_path = component.url
-        if "%(platform_label)" in component.url:
-          download_path = \
-              download_path.replace("%(platform_label)", get_platform_release_label().cdh)
-      wget_and_unpack_package(download_path, file_name, cdh_components_home, False)
-    except Exception:
-      # IMPALA-8517: print backtrace to help debug flaky failures.
-      traceback.print_exc()
-      raise
-
-  execute_many(download, cdh_components)
-
 
-def download_cdp_components(cdp_components, url_prefix):
-  """
-  Downloads and unpacks the CDP components for a given URL prefix into
-  $CDP_COMPONENTS_HOME if not found.
-
-  cdp_components: list of CdpComponent instances
-  """
-  cdp_components_home = os.environ.get("CDP_COMPONENTS_HOME")
-  if not cdp_components_home:
+def create_directory_from_env_var(env_var):
+  dir_name = os.environ.get(env_var)
+  if not dir_name:
     logging.error("Impala environment not set up correctly, make sure "
-                  "$CDP_COMPONENTS_HOME is set.")
+        "{0} is set.".format(env_var))
     sys.exit(1)
+  if not os.path.exists(dir_name):
+    os.makedirs(dir_name)
+
+
+def get_toolchain_downloads():
+  toolchain_packages = []
+  # The LLVM and GCC packages are the largest packages in the toolchain (Kudu is handled
+  # separately). Sort them first so their downloads start as soon as possible.
+  llvm_package = ToolchainPackage("llvm")
+  llvm_package_asserts = ToolchainPackage("llvm", explicit_version="5.0.1-asserts-p1")
+  gcc_package = ToolchainPackage("gcc")
+  toolchain_packages += [llvm_package, llvm_package_asserts, gcc_package]
+  toolchain_packages += map(ToolchainPackage,
+      ["avro", "binutils", "boost", "breakpad", "bzip2", "cctz", "cmake", "crcutil",
+       "flatbuffers", "gdb", "gflags", "glog", "gperftools", "gtest", "libev",
+       "libunwind", "lz4", "openldap", "openssl", "orc", "protobuf",
+       "rapidjson", "re2", "snappy", "thrift", "tpc-h", "tpc-ds", "zlib", "zstd"])
+  toolchain_packages += [ToolchainPackage("thrift",
+      explicit_version=os.environ.get("IMPALA_THRIFT11_VERSION"))]
+  # Check whether this platform is supported (or whether a valid custom toolchain
+  # has been provided).
+  if not try_get_platform_release_label() \
+     or not try_get_platform_release_label().toolchain:
+    toolchain_root = os.environ.get("IMPALA_TOOLCHAIN")
+    # This would throw an exception if the custom toolchain were not valid
+    check_custom_toolchain(toolchain_root, toolchain_packages)
+    # Nothing to download
+    return []
+  return toolchain_packages
 
-  # Create the directory where CDP components live if necessary.
-  if not os.path.exists(cdp_components_home):
-    os.makedirs(cdp_components_home)
 
-  def download(component):
+def get_hadoop_downloads():
+  cluster_components = []
+  use_cdp_hive = os.environ["USE_CDP_HIVE"] == "true"
+  if use_cdp_hive:
+    hadoop = CdpComponent("hadoop")
+    hbase = CdpComponent("hbase", archive_basename_tmpl="hbase-${version}-bin",
+                         unpack_directory_tmpl="hbase-${version}")
+    hive = CdpComponent("hive", archive_basename_tmpl="apache-hive-${version}-bin")
+    hive_src = CdpComponent("hive-source",
+                            explicit_version=os.environ.get("IMPALA_HIVE_VERSION"),
+                            archive_basename_tmpl="hive-${version}-source",
+                            unpack_directory_tmpl="hive-${version}")
+    tez = CdpComponent("tez", archive_basename_tmpl="tez-${version}-minimal",
+                       makedir=True)
+    cluster_components.extend([hadoop, hbase, hive, hive_src, tez])
+  else:
+    cluster_components.extend(map(CdhComponent, ["hadoop", "hbase", "hive"]))
+  # Sentry is always CDH
+  cluster_components.append(CdhComponent("sentry"))
+  # Ranger is always CDP
+  cluster_components.append(CdpComponent("ranger",
+                                         archive_basename_tmpl="ranger-${version}-admin"))
 
-    if os.path.isdir(component.pkg_directory): return
-    file_name = "{0}.tar.gz".format(component.basename)
-    download_path = "{0}/{1}".format(url_prefix, file_name)
-    dst = cdp_components_home
-    if component.makedir:
-      # Download and unpack in a temp directory, which we'll later move into place
-      dst = tempfile.mkdtemp(dir=cdp_components_home)
-    try:
-      wget_and_unpack_package(download_path, file_name, dst, False)
-    except:  # noqa
-      # Clean up any partially-unpacked result.
-      if os.path.isdir(component.pkg_directory):
-        shutil.rmtree(component.pkg_directory)
-      # Clean up any temp directory if we made one
-      if component.makedir:
-        shutil.rmtree(dst)
-      raise
-    if component.makedir:
-      os.rename(dst, component.pkg_directory)
-
-  execute_many(download, cdp_components)
-
-
-if __name__ == "__main__":
-  """Validates the presence of $IMPALA_HOME and $IMPALA_TOOLCHAIN in the environment.-
-  By checking $IMPALA_HOME is set, we assume that IMPALA_{LIB}_VERSION will be set
-  as well. Will create the directory specified by $IMPALA_TOOLCHAIN if it doesn't exist
-  yet. Each of the packages specified in `packages` is downloaded and extracted into
-  $IMPALA_TOOLCHAIN. If $DOWNLOAD_CDH_COMPONENTS is true, the presence of
-  $IMPALA_TOOLCHAIN_HOST and $CDH_BUILD_NUMBER will be checked and this function will also
-  download the following CDH/CDP components into the directory specified by
-  $CDH_COMPONENTS_HOME/$CDP_COMPONENTS_HOME.
-  - hadoop (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDH_BUILD_NUMBER)
-  - hbase (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDH_BUILD_NUMBER)
-  - hive (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDH_BUILD_NUMBER)
-  - sentry (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDH_BUILD_NUMBER)
-  - llama-minikdc (downloaded from $TOOLCHAIN_HOST)
-  - ranger (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDP_BUILD_NUMBER)
-  - hive3 (downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDP_BUILD_NUMBER)
+  # llama-minikdc is used for testing and not something that Impala needs to be built
+  # against. It does not get updated very frequently unlike the other CDH components.
+  # It is stored in a special location compared to other components.
+  toolchain_host = os.environ["IMPALA_TOOLCHAIN_HOST"]
+  download_path_prefix = "https://{0}/build/cdh_components/".format(toolchain_host)
+  destination_basedir = os.environ["CDH_COMPONENTS_HOME"]
+  cluster_components += [EnvVersionedPackage("llama-minikdc", download_path_prefix,
+      destination_basedir)]
+  return cluster_components
+
+
+def get_kudu_downloads(use_kudu_stub):
+  # If Kudu is not supported, we download centos7 kudu to build the kudu stub.
+  # TODO: Should this be from toolchain or CDH? Does it matter?
+  kudu_downloads = []
+  if use_kudu_stub:
+    kudu_downloads += [ToolchainKudu("centos7")]
+  else:
+    use_cdh_kudu = os.getenv("USE_CDH_KUDU") == "true"
+    if use_cdh_kudu:
+      if not try_get_platform_release_label() \
+         or not try_get_platform_release_label().cdh:
+        logging.error("CDH Kudu is not supported on this platform. Set "
+                      "USE_CDH_KUDU=false to use the toolchain Kudu.")
+        sys.exit(1)
+      kudu_downloads += [CdhKudu(get_platform_release_label().cdh)]
+    else:
+      kudu_downloads += [ToolchainKudu()]
+
+  # Independent of the regular Kudu package, there is also a Kudu Java package. This
+  # always needs to be downloaded from the CDH components, because the toolchain
+  # does not produce the Java artifacts.
+  # TODO: Does this make any sense with the Kudu stub?
+  kudu_downloads += [CdhKuduJava()]
+  return kudu_downloads
+
+
+def main():
+  """Validates that bin/impala-config.sh has been sourced by verifying that $IMPALA_HOME
+  and $IMPALA_TOOLCHAIN are in the environment. We assume that if these are set, then
+  IMPALA_<PACKAGE>_VERSION environment variables are also set. This will create the
+  directory specified by $IMPALA_TOOLCHAIN if it does not already exist. Then, it will
+  compute what packages need to be downloaded. Packages are only downloaded if they are
+  not already present in $IMPALA_TOOLCHAIN. There are two main categories of packages.
+  Toolchain packages are native packages built using the native toolchain. These are
+  always downloaded. Hadoop component packages are the CDH or CDP builds of Hadoop
+  components such as Hadoop, Hive, HBase, etc. Hadoop component packages are organized
+  as a consistent set of compatible version via a build number (i.e. CDH_BUILD_NUMBER
+  and CDP_BUILD_NUMBER). Hadoop component packages are only downloaded if
+  $DOWNLOAD_CDH_COMPONENTS is true. CDH Hadoop packages are downloaded into
+  $CDH_COMPONENTS_HOME. CDP Hadoop packages are downloaded into $CDP_COMPONENTS_HOME.
+  The versions used for Hadoop components depend on whether USE_CDP_HIVE is true or
+  false. If true, most components get the CDP versions based on the $CDP_BUILD_NUMBER.
+  If false, most components get the CDH versions based on the $CDH_BUILD_NUMBER.
+  The exceptions are:
+  - sentry (always downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDH_BUILD_NUMBER)
+  - llama-minikdc (always downloaded from $TOOLCHAIN_HOST)
+  - ranger (always downloaded from $IMPALA_TOOLCHAIN_HOST for a given $CDP_BUILD_NUMBER)
+  - kudu (currently always downloaded from $IMPALA_TOOLCHAIN_HOST for a given
+    $CDH_BUILD_NUMBER)
+  If Kudu is not supported on this platform (or KUDU_IS_SUPPORTED=false), then this
+  builds a Kudu stub to allow for compilation without Kudu support.
   """
   logging.basicConfig(level=logging.INFO,
       format='%(asctime)s %(threadName)s %(levelname)s: %(message)s')
@@ -531,88 +702,32 @@ if __name__ == "__main__":
           "impala-config.sh is sourced.")
     sys.exit(1)
 
-  # Create the destination directory if necessary
-  toolchain_root = os.environ.get("IMPALA_TOOLCHAIN")
-  if not toolchain_root:
-    logging.error("Impala environment not set up correctly, make sure "
-          "$IMPALA_TOOLCHAIN is set.")
-    sys.exit(1)
+  # Create the toolchain directory if necessary
+  create_directory_from_env_var("IMPALA_TOOLCHAIN")
 
-  if not os.path.exists(toolchain_root):
-    os.makedirs(toolchain_root)
-
-  use_cdh_kudu = os.getenv("USE_CDH_KUDU") == "true"
-  if os.environ["KUDU_IS_SUPPORTED"] != "true":
-    # We need gcc to build the Kudu stub, so download it first, and we also
-    # need the toolchain Kudu.
-    bootstrap(toolchain_root, [Package("gcc")])
-    use_cdh_kudu = False
-
-  # LLVM and Kudu are the largest packages. Sort them first so that
-  # their download starts as soon as possible.
-  packages = []
-  if not use_cdh_kudu: packages += [Package("kudu")]
-  packages += map(Package, ["llvm",
-      "avro", "binutils", "boost", "breakpad", "bzip2", "cctz", "cmake", "crcutil",
-      "flatbuffers", "gcc", "gdb", "gflags", "glog", "gperftools", "gtest", "libev",
-      "libunwind", "lz4", "openldap", "openssl", "orc", "protobuf",
-      "rapidjson", "re2", "snappy", "thrift", "tpc-h", "tpc-ds", "zlib", "zstd"])
-  packages.insert(0, Package("llvm", "5.0.1-asserts-p1"))
-  packages.insert(0, Package("thrift", os.environ.get("IMPALA_THRIFT11_VERSION")))
-  bootstrap(toolchain_root, packages)
-
-  # Download the CDH components if necessary.
-  if not os.getenv("DOWNLOAD_CDH_COMPONENTS", "false") == "true": sys.exit(0)
-
-  if "IMPALA_TOOLCHAIN_HOST" not in os.environ or "CDH_BUILD_NUMBER" not in os.environ:
-    logging.error("Impala environment not set up correctly, make sure "
-                  "impala-config.sh is sourced.")
-    sys.exit(1)
+  use_kudu_stub = os.environ["KUDU_IS_SUPPORTED"] != "true"
 
-  toolchain_host = os.environ["IMPALA_TOOLCHAIN_HOST"]
-  cdh_build_number = os.environ["CDH_BUILD_NUMBER"]
+  downloads = []
+  downloads += get_toolchain_downloads()
+  kudu_download = None
+  if os.getenv("DOWNLOAD_CDH_COMPONENTS", "false") == "true":
+    create_directory_from_env_var("CDH_COMPONENTS_HOME")
+    create_directory_from_env_var("CDP_COMPONENTS_HOME")
+    downloads += get_kudu_downloads(use_kudu_stub)
+    downloads += get_hadoop_downloads()
 
-  cdh_components = map(Package, ["hbase", "sentry"])
-  use_cdp_hive = os.getenv("USE_CDP_HIVE") == "true"
-  if not use_cdp_hive:
-    cdh_components += [Package("hive"), Package("hadoop")]
+  components_needing_download = [d for d in downloads if d.needs_download()]
 
-  if use_cdh_kudu:
-    if not try_get_platform_release_label() or not try_get_platform_release_label().cdh:
-      logging.error("CDH Kudu is not supported on this platform. Set USE_CDH_KUDU=false "
-                    "to use the toolchain Kudu.")
-      sys.exit(1)
-    cdh_components += [Package("kudu")]
-  # Always download Kudu's jars regardless of USE_CDH_KUDU since they
-  # aren't platform dependent and aren't packaged by the toolchain.
-  cdh_components += [Package("kudu-java")]
-  download_path_prefix = \
-      "https://{0}/build/cdh_components/{1}/tarballs/".format(toolchain_host,
-                                                              cdh_build_number)
-  download_cdh_components(toolchain_root, cdh_components, download_path_prefix)
+  def download(component):
+    component.download()
 
+  execute_many(download, components_needing_download)
 
-  # llama-minikdc is used for testing and not something that Impala needs to be built
-  # against. It does not get updated very frequently unlike the other CDH components.
-  cdh_components = [Package("llama-minikdc")]
-  download_path_prefix = "{0}/cdh_components/".format(TOOLCHAIN_HOST)
-  download_cdh_components(toolchain_root, cdh_components, download_path_prefix)
-
-  cdp_build_number = os.environ["CDP_BUILD_NUMBER"]
-  cdp_components = [
-    CdpComponent("ranger-{0}-admin".format(os.environ.get("IMPALA_RANGER_VERSION"))),
-  ]
-  if use_cdp_hive:
-    hive_version = os.environ.get("IMPALA_HIVE_VERSION")
-    cdp_components.append(CdpComponent("hive-{0}-source".format(hive_version),
-                          pkg_directory="hive-{0}".format(hive_version))),
-    cdp_components.append(CdpComponent("apache-hive-{0}-bin".format(hive_version))),
-    cdp_components.append(CdpComponent("hadoop-{0}"
-                          .format(os.environ.get("IMPALA_HADOOP_VERSION")))),
-    cdp_components.append(CdpComponent(
-        "tez-{0}-minimal".format(os.environ.get("IMPALA_TEZ_VERSION")),
-        makedir=True))
-  download_path_prefix = \
-    "https://{0}/build/cdp_components/{1}/tarballs".format(toolchain_host,
-                                                           cdp_build_number)
-  download_cdp_components(cdp_components, download_path_prefix)
+  if use_kudu_stub:
+    # Find the kudu package directory and the gcc package directory
+    kudu_download = [d for d in downloads if d.name == 'kudu'][0]
+    gcc_download = [d for d in downloads if d.name == 'gcc'][0]
+    build_kudu_stub(kudu_download.pkg_directory(), gcc_download.pkg_directory())
+
+
+if __name__ == "__main__": main()
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 356eceb..d14d76d 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -168,19 +168,28 @@ fi
 export IMPALA_TOOLCHAIN_HOST
 export CDH_MAJOR_VERSION=6
 export CDH_BUILD_NUMBER=1173663
-export CDP_BUILD_NUMBER=1352353
+export CDH_MAVEN_REPOSITORY=\
+"https://${IMPALA_TOOLCHAIN_HOST}/build/cdh_components/${CDH_BUILD_NUMBER}/maven"
 export CDH_HADOOP_VERSION=3.0.0-cdh6.x-SNAPSHOT
+export CDH_HBASE_VERSION=2.1.0-cdh6.x-SNAPSHOT
+export CDH_HIVE_VERSION=2.1.1-cdh6.x-SNAPSHOT
+export CDH_SENTRY_VERSION=2.1.0-cdh6.x-SNAPSHOT
+
+export CDP_BUILD_NUMBER=1352353
+export CDP_MAVEN_REPOSITORY=\
+"https://${IMPALA_TOOLCHAIN_HOST}/build/cdp_components/${CDP_BUILD_NUMBER}/maven"
 export CDP_HADOOP_VERSION=3.1.1.7.1.0.0-33
-export IMPALA_HBASE_VERSION=2.1.0-cdh6.x-SNAPSHOT
-export IMPALA_SENTRY_VERSION=2.1.0-cdh6.x-SNAPSHOT
-export IMPALA_RANGER_VERSION=1.2.0.7.1.0.0-33
+export CDP_HBASE_VERSION=2.2.0.7.1.0.0-33
+export CDP_HIVE_VERSION=3.1.0.7.1.0.0-33
+export CDP_RANGER_VERSION=1.2.0.7.1.0.0-33
+export CDP_TEZ_VERSION=0.9.1.7.1.0.0-33
+export CDP_KNOX_VERSION=1.0.0.7.1.0.0-33
+
 export IMPALA_PARQUET_VERSION=1.9.0-cdh6.x-SNAPSHOT
 export IMPALA_AVRO_JAVA_VERSION=1.8.2-cdh6.x-SNAPSHOT
 export IMPALA_LLAMA_MINIKDC_VERSION=1.0.0
 export IMPALA_KITE_VERSION=1.0.0-cdh6.x-SNAPSHOT
 export IMPALA_KUDU_JAVA_VERSION=1.10.0-cdh6.x-SNAPSHOT
-export CDH_HIVE_VERSION=2.1.1-cdh6.x-SNAPSHOT
-export CDP_HIVE_VERSION=3.1.0.7.1.0.0-33
 
 # When IMPALA_(CDH_COMPONENT)_URL are overridden, they may contain '$(platform_label)'
 # which will be substituted for the CDH platform label in bootstrap_toolchain.py
@@ -200,6 +209,18 @@ if [ -f "$IMPALA_HOME/bin/impala-config-local.sh" ]; then
   . "$IMPALA_HOME/bin/impala-config-local.sh"
 fi
 
+export CDH_HIVE_URL=${CDH_HIVE_URL-}
+export CDH_HADOOP_URL=${CDH_HADOOP_URL-}
+export CDH_HBASE_URL=${CDH_HBASE_URL-}
+export CDH_SENTRY_URL=${CDH_SENTRY_URL-}
+
+export CDP_HIVE_URL=${CDP_HIVE_URL-}
+export CDP_HIVE_SOURCE_URL=${CDP_HIVE_SOURCE_URL-}
+export CDP_HADOOP_URL=${CDP_HADOOP_URL-}
+export CDP_HBASE_URL=${CDP_HBASE_URL-}
+export CDP_TEZ_URL=${CDP_TEZ_URL-}
+export CDP_RANGER_URL=${CDP_RANGER_URL-}
+
 export CDH_COMPONENTS_HOME="$IMPALA_TOOLCHAIN/cdh_components-$CDH_BUILD_NUMBER"
 export CDP_COMPONENTS_HOME="$IMPALA_TOOLCHAIN/cdp_components-$CDP_BUILD_NUMBER"
 export USE_CDP_HIVE=${USE_CDP_HIVE-false}
@@ -207,17 +228,36 @@ if $USE_CDP_HIVE; then
   # When USE_CDP_HIVE is set we use the CDP hive version to build as well as deploy in
   # the minicluster
   export IMPALA_HIVE_VERSION=${CDP_HIVE_VERSION}
-  export IMPALA_TEZ_VERSION=0.9.1.7.1.0.0-33
-  export IMPALA_KNOX_VERSION=1.0.0.7.1.0.0-33
+  export IMPALA_HIVE_URL=${CDP_HIVE_URL-}
+  export IMPALA_HIVE_SOURCE_URL=${CDP_HIVE_SOURCE_URL-}
   export IMPALA_HADOOP_VERSION=${CDP_HADOOP_VERSION}
-  export HADOOP_HOME="$CDP_COMPONENTS_HOME/hadoop-${CDP_HADOOP_VERSION}/"
+  export IMPALA_HADOOP_URL=${CDP_HADOOP_URL-}
+  export IMPALA_HBASE_VERSION=${CDP_HBASE_VERSION}
+  export IMPALA_HBASE_URL=${CDP_HBASE_URL-}
+  export IMPALA_TEZ_VERSION=${CDP_TEZ_VERSION}
+  export IMPALA_TEZ_URL=${CDP_TEZ_URL-}
+  export IMPALA_KNOX_VERSION=${CDP_KNOX_VERSION}
+  export HADOOP_HOME="$CDP_COMPONENTS_HOME/hadoop-${IMPALA_HADOOP_VERSION}/"
 else
   # CDH hive version is used to build and deploy in minicluster when USE_CDP_HIVE is
   # false
   export IMPALA_HIVE_VERSION=${CDH_HIVE_VERSION}
+  export IMPALA_HIVE_URL=${CDH_HIVE_URL-}
   export IMPALA_HADOOP_VERSION=${CDH_HADOOP_VERSION}
+  export IMPALA_HADOOP_URL=${CDH_HADOOP_URL-}
+  export IMPALA_HBASE_VERSION=${CDH_HBASE_VERSION}
+  export IMPALA_HBASE_URL=${CDH_HBASE_URL-}
   export HADOOP_HOME="$CDH_COMPONENTS_HOME/hadoop-${IMPALA_HADOOP_VERSION}/"
 fi
+
+# Ranger always uses the CDP version
+export IMPALA_RANGER_VERSION=${CDP_RANGER_VERSION}
+export IMPALA_RANGER_URL=${CDP_RANGER_URL-}
+
+# Sentry always uses the CDH version
+export IMPALA_SENTRY_VERSION=${CDH_SENTRY_VERSION}
+export IMPALA_SENTRY_URL=${CDH_SENTRY_URL-}
+
 # Extract the first component of the hive version.
 # Allow overriding of Hive source location in case we want to build Impala without
 # a complete Hive build. This is used by fe/pom.xml to activate compatibility shims
@@ -320,17 +360,19 @@ if $USE_CDP_HIVE; then
 ${IMPALA_HIVE_VERSION}"}
   # Set the path to the hive_metastore.thrift which is used to build thrift code
   export HIVE_METASTORE_THRIFT_DIR=$HIVE_SRC_DIR/standalone-metastore/src/main/thrift
+  export TEZ_HOME="$CDP_COMPONENTS_HOME/tez-${IMPALA_TEZ_VERSION}-minimal"
+  export HBASE_HOME="$CDP_COMPONENTS_HOME/hbase-${IMPALA_HBASE_VERSION}/"
   # It is likely that devs will want to work with both the versions of metastore
   # if cdp hive is being used change the metastore db name, so we don't have to
   # format the metastore db everytime we switch between hive versions
   export METASTORE_DB=${METASTORE_DB-"$(cut -c-59 <<< HMS$ESCAPED_IMPALA_HOME)_cdp"}
-  export TEZ_HOME="$CDP_COMPONENTS_HOME/tez-${IMPALA_TEZ_VERSION}-minimal"
 else
   export HIVE_HOME="$CDH_COMPONENTS_HOME/hive-${IMPALA_HIVE_VERSION}"
   # Allow overriding of Hive source location in case we want to build Impala without
 # a complete Hive build.
   export HIVE_SRC_DIR=${HIVE_SRC_DIR_OVERRIDE:-"${HIVE_HOME}/src"}
   export HIVE_METASTORE_THRIFT_DIR=$HIVE_SRC_DIR/metastore/if
+  export HBASE_HOME="$CDH_COMPONENTS_HOME/hbase-${IMPALA_HBASE_VERSION}/"
   export METASTORE_DB=${METASTORE_DB-$(cut -c-63 <<< HMS$ESCAPED_IMPALA_HOME)}
 fi
 # Set the Hive binaries in the path
@@ -593,7 +635,6 @@ export AUX_CLASSPATH="${LZO_JAR_PATH}"
 ### Tell hive not to use jline
 export HADOOP_USER_CLASSPATH_FIRST=true
 
-export HBASE_HOME="$CDH_COMPONENTS_HOME/hbase-${IMPALA_HBASE_VERSION}/"
 export PATH="$HBASE_HOME/bin:$PATH"
 
 # Add the jars so hive can create hbase tables.
diff --git a/impala-parent/pom.xml b/impala-parent/pom.xml
index 46af7d8..13cf973 100644
--- a/impala-parent/pom.xml
+++ b/impala-parent/pom.xml
@@ -160,7 +160,7 @@ under the License.
       CDP does not use maven SNAPSHOT versions - every build has a version number.
       -->
       <id>impala.cdp.repo</id>
-      <url>https://${env.IMPALA_TOOLCHAIN_HOST}/build/cdp_components/${env.CDP_BUILD_NUMBER}/maven</url>
+      <url>${env.CDP_MAVEN_REPOSITORY}</url>
       <name>Impala CDP Repository</name>
       <snapshots>
         <enabled>false</enabled>
@@ -172,7 +172,7 @@ under the License.
       CDH uses maven SNAPSHOT versions for non-released versions of components.
       -->
       <id>impala.cdh.repo</id>
-      <url>https://${env.IMPALA_TOOLCHAIN_HOST}/build/cdh_components/${env.CDH_BUILD_NUMBER}/maven</url>
+      <url>${env.CDH_MAVEN_REPOSITORY}</url>
       <name>Impala CDH Repository</name>
       <snapshots>
         <enabled>true</enabled>