You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2016/05/24 23:41:05 UTC

[1/2] incubator-impala git commit: Revert "Revert "Add Kudu test helpers""

Repository: incubator-impala
Updated Branches:
  refs/heads/master d70ffa455 -> 5112e65be


Revert "Revert "Add Kudu test helpers""

This reverts commit f8dd5413b65d30646c3745dfc738ed812d50a51f and
effectively re-adds commit 9248dcb70478b8f93f022893776a0960f45fdc28. The
difference between this patch and its original is that I fixed the
changes introduced in infra/python/bootstrap_virtualenv.py to be
python2.4-compatible:

- removed the use of str.format(), preferring a str.join() pattern
- removed the call of the exit() builtin to prefer sys.exit()

The only testing I did for this patch was to ensure
CDH Impala-packaging-on-demand works.

Change-Id: I02ed97473868eacf45b25abe89b41e6fa2fce325
Reviewed-on: http://gerrit.cloudera.org:8080/3160
Reviewed-by: Michael Brown <mi...@cloudera.com>
Tested-by: Internal Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/5112e65b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/5112e65b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/5112e65b

Branch: refs/heads/master
Commit: 5112e65be2c191cd606396f853ce51e3f915f097
Parents: f0e0286
Author: Michael Brown <mi...@cloudera.com>
Authored: Fri May 20 10:31:52 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue May 24 16:40:59 2016 -0700

----------------------------------------------------------------------
 bin/impala-ipython                   |   9 +-
 bin/impala-py.test                   |  17 +--
 bin/impala-python                    |   4 +-
 bin/impala-python-common.sh          |  25 +++++
 infra/python/bootstrap_virtualenv.py | 154 ++++++++++++++++++++++++---
 infra/python/deps/requirements.txt   |  15 +++
 tests/common/__init__.py             |   3 +-
 tests/conftest.py                    | 167 +++++++++++++++++++++++++++++-
 8 files changed, 350 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/bin/impala-ipython
----------------------------------------------------------------------
diff --git a/bin/impala-ipython b/bin/impala-ipython
index 8833384..e7f9806 100755
--- a/bin/impala-ipython
+++ b/bin/impala-ipython
@@ -1,10 +1,3 @@
 #!/bin/bash
-
-set -eu -o pipefail
-
-PY_DIR=$(dirname "$0")/../infra/python
-
-# impala-python will build or upgrade the python virtualenv automatically.
-impala-python -c ""
-
+source $(dirname "$0")/impala-python-common.sh
 exec "$PY_DIR/env/bin/ipython" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/bin/impala-py.test
----------------------------------------------------------------------
diff --git a/bin/impala-py.test b/bin/impala-py.test
index 55756a0..48d01e9 100755
--- a/bin/impala-py.test
+++ b/bin/impala-py.test
@@ -1,14 +1,3 @@
-#!/usr/bin/env impala-python
-# EASY-INSTALL-ENTRY-SCRIPT: 'pytest==2.7.2','console_scripts','py.test'
-
-# This was copied straight from py.test, the only modification is to use impala-python
-# instead of python above
-
-__requires__ = 'pytest==2.7.2'
-import sys
-from pkg_resources import load_entry_point
-
-if __name__ == '__main__':
-    sys.exit(
-        load_entry_point('pytest==2.7.2', 'console_scripts', 'py.test')()
-    )
+#!/bin/bash
+source $(dirname "$0")/impala-python-common.sh
+exec "$PY_DIR/env/bin/py.test" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/bin/impala-python
----------------------------------------------------------------------
diff --git a/bin/impala-python b/bin/impala-python
index 67c7be3..5ff4190 100755
--- a/bin/impala-python
+++ b/bin/impala-python
@@ -1,5 +1,3 @@
 #!/bin/bash
-set -eu -o pipefail
-PY_DIR=$(dirname "$0")/../infra/python
-python "$PY_DIR/bootstrap_virtualenv.py"
+source $(dirname "$0")/impala-python-common.sh
 exec "$PY_DIR/env/bin/python" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/bin/impala-python-common.sh
----------------------------------------------------------------------
diff --git a/bin/impala-python-common.sh b/bin/impala-python-common.sh
new file mode 100644
index 0000000..298a0ca
--- /dev/null
+++ b/bin/impala-python-common.sh
@@ -0,0 +1,25 @@
+# Copyright 2016 Cloudera Inc.
+#
+# Licensed 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.
+
+# This file is intended to be sourced to perform common setup for
+# $IMPALA_HOME/bin/impala-py* executables.
+
+set -euo pipefail
+trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
+
+LD_LIBRARY_PATH+=":$(python $IMPALA_HOME/infra/python/bootstrap_virtualenv.py \
+  --print-ld-library-path)"
+
+PY_DIR=$(dirname "$0")/../infra/python
+python "$PY_DIR/bootstrap_virtualenv.py"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/infra/python/bootstrap_virtualenv.py
----------------------------------------------------------------------
diff --git a/infra/python/bootstrap_virtualenv.py b/infra/python/bootstrap_virtualenv.py
index 11065db..b0ebb83 100644
--- a/infra/python/bootstrap_virtualenv.py
+++ b/infra/python/bootstrap_virtualenv.py
@@ -26,6 +26,7 @@ import optparse
 import os
 import shutil
 import subprocess
+import sys
 import tarfile
 import tempfile
 import textwrap
@@ -62,17 +63,28 @@ def create_virtualenv():
   shutil.rmtree(build_dir)
 
 
-def exec_cmd(args):
+def exec_cmd(args, **kwargs):
   '''Executes a command and waits for it to finish, raises an exception if the return
-     status is not zero.
+     status is not zero. The command output is returned.
 
-     'args' uses the same format as subprocess.Popen().
+     'args' and 'kwargs' use the same format as subprocess.Popen().
   '''
-  process = subprocess.Popen(args, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+  process = subprocess.Popen(args, stdout=subprocess.PIPE, stderr=subprocess.STDOUT,
+      **kwargs)
   output = process.communicate()[0]
   if process.returncode != 0:
     raise Exception("Command returned non-zero status\nCommand: %s\nOutput: %s"
         % (args, output))
+  return output
+
+
+def exec_pip_install(args, **popen_kwargs):
+  # Don't call the virtualenv pip directly, it uses a hashbang to to call the python
+  # virtualenv using an absolute path. If the path to the virtualenv is very long, the
+  # hashbang won't work.
+  exec_cmd([os.path.join(ENV_DIR, "bin", "python"), os.path.join(ENV_DIR, "bin", "pip"),
+    "install", "--no-index", "--find-links",
+    "file://%s" % urllib.pathname2url(os.path.abspath(DEPS_DIR))] + args, **popen_kwargs)
 
 
 def find_file(*paths):
@@ -108,19 +120,119 @@ def detect_python_cmd():
 
 
 def install_deps():
-  LOG.info("Installing packages into virtualenv")
-  # Don't call the virtualenv pip directly, it uses a hashbang to to call the python
-  # virtualenv using an absolute path. If the path to the virtualenv is very long, the
-  # hashbang won't work.
-  # --no-cache-dir is used because the dev version of Impyla may be the same even though
-  # the contents are different. Since the version doesn't change, pip may use its cached
-  # build.
-  exec_cmd([os.path.join(ENV_DIR, "bin", "python"), os.path.join(ENV_DIR, "bin", "pip"),
-    "install", "--no-cache-dir", "--no-index", "--find-links",
-    "file://%s" % urllib.pathname2url(os.path.abspath(DEPS_DIR)), "-r", REQS_PATH])
+  LOG.info("Installing packages into the virtualenv")
+  exec_pip_install(["-r", REQS_PATH])
   shutil.copyfile(REQS_PATH, INSTALLED_REQS_PATH)
 
 
+def install_kudu_client_if_possible():
+  """Installs the Kudu python module if possible. The Kudu module is the only one that
+     requires the toolchain. If the toolchain isn't in use or hasn't been populated
+     yet, nothing will be done. Also nothing will be done if the Kudu client lib required
+     by the module isn't available (as determined by KUDU_IS_SUPPORTED).
+  """
+  if os.environ["KUDU_IS_SUPPORTED"] != "true":
+    LOG.debug("Skipping Kudu: Kudu is not supported")
+    return
+  impala_toolchain_dir = os.environ.get("IMPALA_TOOLCHAIN")
+  if not impala_toolchain_dir:
+    LOG.debug("Skipping Kudu: IMPALA_TOOLCHAIN not set")
+    return
+  toolchain_kudu_dir = os.path.join(
+      impala_toolchain_dir, "kudu-" + os.environ["IMPALA_KUDU_VERSION"])
+  if not os.path.exists(toolchain_kudu_dir):
+    LOG.debug("Skipping Kudu: %s doesn't exist" % toolchain_kudu_dir)
+    return
+
+  # The "pip" command could be used to provide the version of Kudu installed (if any)
+  # but it's a little too slow. Running the virtualenv python to detect the installed
+  # version is faster.
+  actual_version_string = exec_cmd([os.path.join(ENV_DIR, "bin", "python"), "-c",
+      textwrap.dedent("""
+      try:
+        import kudu
+        print kudu.__version__
+      except ImportError:
+        pass""")]).strip()
+  actual_version = [int(v) for v in actual_version_string.split(".") if v]
+
+  reqs_file = open(REQS_PATH)
+  try:
+    for line in reqs_file:
+      if not line.startswith("# kudu-python=="):
+        continue
+      expected_version_string = line.split()[1].split("==")[1]
+      break
+    else:
+      raise Exception("Unable to find kudu-python version in requirements file")
+  finally:
+    reqs_file.close()
+  expected_version = [int(v) for v in expected_version_string.split(".")]
+
+  if actual_version and actual_version == expected_version:
+    LOG.debug("Skipping Kudu: Installed %s == required %s"
+        % (actual_version_string, expected_version_string))
+    return
+  LOG.debug("Kudu installation required. Actual version %s. Required version %s.",
+      actual_version, expected_version)
+
+  LOG.info("Installing Kudu into the virtualenv")
+  # The installation requires that KUDU_HOME/build/latest exists. An empty directory
+  # structure will be made to satisfy that. The Kudu client headers and lib will be made
+  # available through GCC environment variables.
+  fake_kudu_build_dir = os.path.join(tempfile.gettempdir(), "virtualenv-kudu")
+  try:
+    artifact_dir = os.path.join(fake_kudu_build_dir, "build", "latest")
+    if not os.path.exists(artifact_dir):
+      os.makedirs(artifact_dir)
+    env = dict(os.environ)
+    env["KUDU_HOME"] = fake_kudu_build_dir
+    kudu_client_dir = find_kudu_client_install_dir()
+    env["CPLUS_INCLUDE_PATH"] = os.path.join(kudu_client_dir, "include")
+    env["LIBRARY_PATH"] = os.path.pathsep.join([os.path.join(kudu_client_dir, 'lib'),
+                                                os.path.join(kudu_client_dir, 'lib64')])
+
+    exec_pip_install(["kudu-python==" + expected_version_string], env=env)
+  finally:
+    try:
+      shutil.rmtree(fake_kudu_build_dir)
+    except Exception:
+      LOG.debug("Error removing temp Kudu build dir", exc_info=True)
+
+
+def find_kudu_client_install_dir():
+  custom_client_dir = os.environ["KUDU_CLIENT_DIR"]
+  if custom_client_dir:
+    install_dir = os.path.join(custom_client_dir, "usr", "local")
+    error_if_kudu_client_not_found(install_dir)
+  else:
+    # If the toolchain appears to have been setup already, then the Kudu client is
+    # required to exist. It's possible that the toolchain won't be setup yet though
+    # since the toolchain bootstrap script depends on the virtualenv.
+    kudu_base_dir = os.path.join(os.environ["IMPALA_TOOLCHAIN"],
+        "kudu-%s" % os.environ["IMPALA_KUDU_VERSION"])
+    install_dir = os.path.join(kudu_base_dir, "debug")
+    if os.path.exists(kudu_base_dir):
+      error_if_kudu_client_not_found(install_dir)
+  return install_dir
+
+
+def error_if_kudu_client_not_found(install_dir):
+  header_path = os.path.join(install_dir, "include", "kudu", "client", "client.h")
+  if not os.path.exists(header_path):
+    raise Exception("Kudu client header not found at %s" % header_path)
+
+  kudu_client_lib = "libkudu_client.so"
+  lib_dir = os.path.join(install_dir, "lib64")
+  if not os.path.exists(lib_dir):
+    lib_dir = os.path.join(install_dir, "lib")
+  for _, _, files in os.walk(lib_dir):
+    for file in files:
+      if file == kudu_client_lib:
+        return
+  raise Exception("%s not found at %s" % (kudu_client_lib, lib_dir))
+
+
 def deps_are_installed():
   if not os.path.exists(INSTALLED_REQS_PATH):
     return False
@@ -148,11 +260,23 @@ def setup_virtualenv_if_not_exists():
 
 
 if __name__ == "__main__":
-  logging.basicConfig(level=logging.INFO)
   parser = optparse.OptionParser()
+  parser.add_option("-l", "--log-level", default="INFO",
+      choices=("DEBUG", "INFO", "WARN", "ERROR"))
   parser.add_option("-r", "--rebuild", action="store_true", help="Force a rebuild of"
       " the virtualenv even if it exists and appears to be completely up-to-date.")
+  parser.add_option("--print-ld-library-path", action="store_true", help="Print the"
+      " LD_LIBRARY_PATH that should be used when running python from the virtualenv.")
   options, args = parser.parse_args()
+
+  if options.print_ld_library_path:
+    kudu_client_dir = find_kudu_client_install_dir()
+    print os.path.pathsep.join([os.path.join(kudu_client_dir, 'lib'),
+                                os.path.join(kudu_client_dir, 'lib64')])
+    sys.exit()
+
+  logging.basicConfig(level=getattr(logging, options.log_level))
   if options.rebuild:
     delete_virtualenv_if_exist()
   setup_virtualenv_if_not_exists()
+  install_kudu_client_if_possible()

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/infra/python/deps/requirements.txt
----------------------------------------------------------------------
diff --git a/infra/python/deps/requirements.txt b/infra/python/deps/requirements.txt
index 2725344..5409027 100644
--- a/infra/python/deps/requirements.txt
+++ b/infra/python/deps/requirements.txt
@@ -53,5 +53,20 @@ requests == 2.7.0
 sh == 1.11
 sqlparse == 0.1.15
 texttable == 0.8.3
+
+# kudu-python is needed but cannot be listed as usual. The Kudu client lib (.so file)
+# is needed for compilation/installation but the client lib is provided by the toolchain.
+# The virtualenv may need to be functional even if the toolchain isn't present. The
+# bootstap_virtualenv.py script special-cases kudu-python, the line below is actually
+# functional and determines the expected kudu-python version. The version must be listed
+# in the format below including # and spacing. Keep this formatting!
+# kudu-python==0.1.1
+  Cython == 0.23.4
+  numpy == 1.10.4
+  # These should eventually be removed  https://issues.apache.org/jira/browse/KUDU-1456
+  unittest2 == 1.1.0
+    linecache2 == 1.0.0
+    traceback2 == 1.4.0
+
 # For dev purposes, not used in scripting. Version 1.2.1 is the latest that supports 2.6.
 ipython == 1.2.1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/tests/common/__init__.py
----------------------------------------------------------------------
diff --git a/tests/common/__init__.py b/tests/common/__init__.py
index 946a474..010f973 100644
--- a/tests/common/__init__.py
+++ b/tests/common/__init__.py
@@ -1 +1,2 @@
-# This file is needed to make the files in this directory a python module
+KUDU_MASTER_HOST = "127.0.0.1"
+KUDU_MASTER_PORT = 7051

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5112e65b/tests/conftest.py
----------------------------------------------------------------------
diff --git a/tests/conftest.py b/tests/conftest.py
index c6a22c0..3fb7f04 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -1,17 +1,21 @@
 # Copyright (c) 2012 Cloudera, Inc. All rights reserved.
 # py.test configuration module
 #
+from impala.dbapi import connect as impala_connect
+from kudu import connect as kudu_connect
+from random import choice, sample
+from string import ascii_lowercase, digits
+from zlib import crc32
+import contextlib
 import logging
 import os
 import pytest
 
-from zlib import crc32
-
+from common import KUDU_MASTER_HOST, KUDU_MASTER_PORT
 from common.test_result_verifier import QueryTestResult
 from tests.common.patterns import is_valid_impala_identifier
 from tests.util.filesystem_utils import FILESYSTEM, ISILON_WEBHDFS_PORT
 
-
 logging.basicConfig(level=logging.INFO, format='%(threadName)s: %(message)s')
 LOG = logging.getLogger('test_configuration')
 
@@ -203,6 +207,9 @@ def unique_database(request, testid_checksum):
 
     name_prefix: string (defaults to test function __name__) - prefix to be used for the
     database name
+
+  For a similar DB-API 2 compliant connection/cursor that use HS2 see the 'conn' and
+  'unique_cursor' fixtures below.
   """
 
   # Test cases are at the function level, so no one should "accidentally" re-scope this.
@@ -235,3 +242,157 @@ def unique_database(request, testid_checksum):
   LOG.info('Created database "{0}" for test ID "{1}"'.format(db_name,
                                                              str(request.node.nodeid)))
   return db_name
+
+
+@pytest.yield_fixture
+def kudu_client():
+  """Provides a new Kudu client as a pytest fixture. The client only exists for the
+     duration of the method it is used in.
+  """
+  kudu_client = kudu_connect(KUDU_MASTER_HOST, KUDU_MASTER_PORT)
+  try:
+    yield kudu_client
+  finally:
+    try:
+      kudu_client.close()
+    except Exception as e:
+      LOG.warn("Error closing Kudu client: %s", e)
+
+
+@pytest.yield_fixture(scope="class")
+def conn(request):
+  """Provides a new DB-API compliant connection to Impala as a pytest fixture. The
+     same connection is used for all test methods in a class. The class may provide the
+     following customizations:
+       - get_db_name(): The name of the database to connect to.
+       - auto_create_db(): If declared and the method returns True, the database will
+         be created before tests run and dropped afterwards. If a database name is
+         provided by get_db_name(), it must not exist. Classes that use both
+         auto_create_db() and get_db_name() should generate a random name in
+         get_db_name() and cache it.
+     The returned connection will have a 'db_name' property.
+
+     See the 'unique_database' fixture above if you want to use Impala's custom python
+     API instead of DB-API.
+  """
+  db_name = __call_cls_method_if_exists(request.cls, "get_db_name")
+  use_unique_conn = __call_cls_method_if_exists(request.cls, "auto_create_db")
+  if use_unique_conn:
+    with __unique_conn(db_name=db_name) as conn:
+      yield conn
+  else:
+    with __auto_closed_conn(db_name=db_name) as conn:
+      yield conn
+
+
+def __call_cls_method_if_exists(cls, method_name):
+  """Returns the result of calling the method 'method_name' on class 'class' if the class
+     defined such a method, otherwise returns None.
+  """
+  method = getattr(cls, method_name, None)
+  if method:
+    return method()
+
+
+@contextlib.contextmanager
+def __unique_conn(db_name=None):
+  """Connects to Impala and creates a new database, then returns a connection to it.
+     This is intended to be used in a "with" block. Upon exit, the database will be
+     dropped. A database name can be provided by 'db_name', a database by that name
+     must not exist prior to calling this method.
+
+     with __unique_conn() as conn:
+       # Use conn
+     # The database no longer exists and the conn is closed.
+
+     The returned connection will have a 'db_name' property.
+  """
+  if not db_name:
+    db_name = choice(ascii_lowercase) + "".join(sample(ascii_lowercase + digits, 5))
+  with __auto_closed_conn() as conn:
+    with __auto_closed_cursor(conn) as cur:
+      cur.execute("CREATE DATABASE %s" % db_name)
+  with __auto_closed_conn(db_name=db_name) as conn:
+    try:
+      yield conn
+    finally:
+      try:
+        with __auto_closed_cursor(conn) as cur:
+          try:
+            cur.execute("USE DEFAULT")
+            cur.execute("DROP DATABASE IF EXISTS %s CASCADE" % db_name)
+          except Exception as e:
+            LOG.warn("Error dropping database: %s", e)
+      except Exception as e:
+        LOG.warn("Error creating a cursor: %s", e)
+
+
+@contextlib.contextmanager
+def __auto_closed_conn(db_name=None):
+  """Returns a connection to Impala. This is intended to be used in a "with" block. The
+     connection will be closed upon exiting the block.
+
+     The returned connection will have a 'db_name' property.
+  """
+  conn = impala_connect(database=db_name)
+  try:
+    conn.db_name = db_name
+    yield conn
+  finally:
+    try:
+      conn.close()
+    except Exception as e:
+      LOG.warn("Error closing Impala connection: %s", e)
+
+
+@pytest.yield_fixture
+def cursor(conn):
+  """Provides a new DB-API compliant cursor from a connection provided by the conn()
+     fixture. The cursor only exists for the duration of the method it is used in.
+
+     The returned cursor will have a 'conn' property. The 'conn' will have a 'db_name'
+     property.
+  """
+  with __auto_closed_cursor(conn) as cur:
+    yield cur
+
+@pytest.yield_fixture(scope="class")
+def cls_cursor(conn):
+  """Provides a new DB-API compliant cursor from a connection provided by the conn()
+     fixture. The cursor exists for the duration of the class it is used in.
+
+     The returned cursor will have a 'conn' property. The 'conn' will have a 'db_name'
+     property.
+  """
+  with __auto_closed_cursor(conn) as cur:
+    yield cur
+
+
+@pytest.yield_fixture
+def unique_cursor():
+  """Provides a new DB-API compliant cursor to a newly created Impala database. The
+     cursor only exists for the duration of the method it is used in. The database will
+     be dropped after the test executes.
+
+     The returned cursor will have a 'conn' property. The 'conn' will have a 'db_name'
+     property.
+  """
+  with __unique_conn() as conn:
+    with __auto_closed_cursor(conn) as cur:
+      yield cur
+
+
+@contextlib.contextmanager
+def __auto_closed_cursor(conn):
+  """Returns a cursor created from conn. This is intended to be used in a "with" block.
+     The cursor will be closed upon exiting the block.
+  """
+  cursor = conn.cursor()
+  cursor.conn = conn
+  try:
+    yield cursor
+  finally:
+    try:
+      cursor.close()
+    except Exception as e:
+      LOG.warn("Error closing Impala cursor: %s", e)


[2/2] incubator-impala git commit: Bump Impala version to 2.7.0

Posted by ta...@apache.org.
Bump Impala version to 2.7.0

Change-Id: Ibf67f61936260e66a5bb0d7fe63e4930850252c6
Reviewed-on: http://gerrit.cloudera.org:8080/3167
Reviewed-by: Bharath Vissapragada <bh...@cloudera.com>
Tested-by: Internal Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/f0e02867
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/f0e02867
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/f0e02867

Branch: refs/heads/master
Commit: f0e02867787aebbbb8dc893d262bde5bb642c35b
Parents: d70ffa4
Author: Harrison Sheinblatt <hs...@cloudera.com>
Authored: Sun May 22 13:22:10 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue May 24 16:40:59 2016 -0700

----------------------------------------------------------------------
 bin/save-version.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f0e02867/bin/save-version.sh
----------------------------------------------------------------------
diff --git a/bin/save-version.sh b/bin/save-version.sh
index 19573e7..1da16ff 100755
--- a/bin/save-version.sh
+++ b/bin/save-version.sh
@@ -17,7 +17,7 @@
 # Note: for internal (aka pre-release) versions, the version should have
 # "-INTERNAL" appended. Parts of the code will look for this to distinguish
 # between released and internal versions.
-VERSION=2.6.0-cdh5-INTERNAL
+VERSION=2.7.0-cdh5-INTERNAL
 GIT_HASH=$(git rev-parse HEAD)
 BUILD_TIME=`date`
 HEADER="# Generated version information from save-version.sh"