You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/08/21 14:53:18 UTC

[arrow] branch master updated: ARROW-3050: [C++] Adopt HiveServer2 client codebase from cloudera/hs2client. Add Thrift to thirdparty toolchain

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

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


The following commit(s) were added to refs/heads/master by this push:
     new dbf531b  ARROW-3050: [C++] Adopt HiveServer2 client codebase from cloudera/hs2client. Add Thrift to thirdparty toolchain
dbf531b is described below

commit dbf531b17bd8f706f83192b4cf4f16be08047716
Author: Wes McKinney <we...@apache.org>
AuthorDate: Wed Aug 15 21:04:40 2018 -0400

    ARROW-3050: [C++] Adopt HiveServer2 client codebase from
    cloudera/hs2client. Add Thrift to thirdparty toolchain
    
    This patch incorporates patches developed at cloudera/hs2client (Apache 2.0) by
    the following authors:
    
    * 12  Wes McKinney <we...@apache.org>, <we...@cloudera.com>
    *  2  Thomas Tauber-Marshall <tm...@cloudera.com>
    *  2  陈晓发 <ch...@chelun.com>
    *  2  Matthew Jacobs <ja...@gmail.com>, <mj...@cloudera.com>
    *  1  Miki Tebeka <mi...@gmail.com>
    *  1  Tim Armstrong <ta...@cloudera.com>
    *  1  henryr <he...@gmail.com>
    
    Closes #2444
    
    Change-Id: I88aed528a9f4d2069a4908f6a09230ade2fbe50a
---
 LICENSE.txt                                        |   42 +
 cpp/CMakeLists.txt                                 |    8 +
 cpp/cmake_modules/ThirdpartyToolchain.cmake        |  144 +++
 cpp/src/arrow/dbi/README.md                        |   24 +
 cpp/src/arrow/dbi/hiveserver2/CMakeLists.txt       |  110 ++
 cpp/src/arrow/dbi/hiveserver2/api.h                |   27 +
 cpp/src/arrow/dbi/hiveserver2/columnar-row-set.cc  |  100 ++
 cpp/src/arrow/dbi/hiveserver2/columnar-row-set.h   |  155 +++
 cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc  |  462 ++++++++
 cpp/src/arrow/dbi/hiveserver2/operation.cc         |  150 +++
 cpp/src/arrow/dbi/hiveserver2/operation.h          |  127 ++
 cpp/src/arrow/dbi/hiveserver2/public-api-test.cc   |   26 +
 cpp/src/arrow/dbi/hiveserver2/sample-usage.cc      |  139 +++
 cpp/src/arrow/dbi/hiveserver2/service.cc           |  113 ++
 cpp/src/arrow/dbi/hiveserver2/service.h            |  140 +++
 cpp/src/arrow/dbi/hiveserver2/session.cc           |  105 ++
 cpp/src/arrow/dbi/hiveserver2/session.h            |   84 ++
 cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc   |  303 +++++
 cpp/src/arrow/dbi/hiveserver2/thrift-internal.h    |   91 ++
 cpp/src/arrow/dbi/hiveserver2/thrift/.gitignore    |    1 +
 .../arrow/dbi/hiveserver2/thrift/CMakeLists.txt    |  110 ++
 .../arrow/dbi/hiveserver2/thrift/ExecStats.thrift  |  103 ++
 .../dbi/hiveserver2/thrift/ImpalaService.thrift    |  300 +++++
 cpp/src/arrow/dbi/hiveserver2/thrift/Status.thrift |   23 +
 .../dbi/hiveserver2/thrift/TCLIService.thrift      | 1180 +++++++++++++++++++
 cpp/src/arrow/dbi/hiveserver2/thrift/Types.thrift  |  218 ++++
 .../arrow/dbi/hiveserver2/thrift/beeswax.thrift    |  174 +++
 cpp/src/arrow/dbi/hiveserver2/thrift/fb303.thrift  |  112 ++
 .../dbi/hiveserver2/thrift/generate_error_codes.py |  293 +++++
 .../dbi/hiveserver2/thrift/hive_metastore.thrift   | 1214 ++++++++++++++++++++
 cpp/src/arrow/dbi/hiveserver2/types.cc             |   45 +
 cpp/src/arrow/dbi/hiveserver2/types.h              |  131 +++
 cpp/src/arrow/dbi/hiveserver2/util.cc              |  251 ++++
 cpp/src/arrow/dbi/hiveserver2/util.h               |   36 +
 cpp/src/arrow/status.h                             |    8 +
 cpp/thirdparty/download_dependencies.sh            |    3 +
 cpp/thirdparty/versions.txt                        |    1 +
 dev/docker-compose.yml                             |   14 +
 dev/docker_common/Dockerfile.xenial.base           |   61 +
 dev/docker_common/wait-for-it.sh                   |  199 ++++
 dev/hiveserver2/Dockerfile                         |   23 +
 dev/hiveserver2/hiveserver2.sh                     |   67 ++
 42 files changed, 6917 insertions(+)

diff --git a/LICENSE.txt b/LICENSE.txt
index 9a7ad1f..e557b80 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -630,3 +630,45 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 --------------------------------------------------------------------------------
+
+This project includes code from the hs2client
+
+https://github.com/cloudera/hs2client
+
+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.
+
+--------------------------------------------------------------------------------
+
+The script dev/docker_common/wait-for-it.sh has the following license
+
+Copyright (c) 2016 Giles Hall
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index bc080aa..9f41142 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -203,6 +203,10 @@ Pass multiple labels by dividing with semicolons")
     "Build Arrow libraries with install_name set to @rpath"
     ON)
 
+  option(ARROW_HIVESERVER2
+    "Build the HiveServer2 client and Arrow adapter"
+    OFF)
+
   option(ARROW_PLASMA
     "Build the plasma object store along with Arrow"
     OFF)
@@ -721,3 +725,7 @@ add_subdirectory(src/arrow)
 if(ARROW_PYTHON)
   add_subdirectory(src/arrow/python)
 endif()
+
+if(ARROW_HIVESERVER2)
+  add_subdirectory(src/arrow/dbi/hiveserver2)
+endif()
diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake
index dd81ee5..124a8d3 100644
--- a/cpp/cmake_modules/ThirdpartyToolchain.cmake
+++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake
@@ -31,6 +31,7 @@ if (NOT "$ENV{ARROW_BUILD_TOOLCHAIN}" STREQUAL "")
   set(BROTLI_HOME "$ENV{ARROW_BUILD_TOOLCHAIN}")
   set(LZ4_HOME "$ENV{ARROW_BUILD_TOOLCHAIN}")
   set(ZSTD_HOME "$ENV{ARROW_BUILD_TOOLCHAIN}")
+  set(THRIFT_HOME "$ENV{ARROW_BUILD_TOOLCHAIN}")
 
   if (NOT DEFINED ENV{BOOST_ROOT})
     # Since we have to set this in the environment, we check whether
@@ -79,6 +80,10 @@ if (DEFINED ENV{PROTOBUF_HOME})
   set(PROTOBUF_HOME "$ENV{PROTOBUF_HOME}")
 endif()
 
+if (DEFINED ENV{THRIFT_HOME})
+  set(THRIFT_HOME "$ENV{THRIFT_HOME}")
+endif()
+
 # ----------------------------------------------------------------------
 # Versions and URLs for toolchain builds, which also can be used to configure
 # offline builds
@@ -192,6 +197,12 @@ else()
   set(ORC_SOURCE_URL "https://github.com/apache/orc/archive/rel/release-${ORC_VERSION}.tar.gz")
 endif()
 
+if (DEFINED ENV{ARROW_THRIFT_URL})
+  set(THRIFT_SOURCE_URL "$ENV{ARROW_THRIFT_URL}")
+else()
+  set(THRIFT_SOURCE_URL "http://archive.apache.org/dist/thrift/${THRIFT_VERSION}/thrift-${THRIFT_VERSION}.tar.gz")
+endif()
+
 # ----------------------------------------------------------------------
 # ExternalProject options
 
@@ -1084,3 +1095,136 @@ if (ARROW_ORC)
   endif()
 
 endif()
+
+# ----------------------------------------------------------------------
+# Thrift
+
+if (ARROW_HIVESERVER2)
+
+# find thrift headers and libs
+find_package(Thrift)
+
+if (NOT THRIFT_FOUND)
+  set(ZLIB_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/zlib_ep/src/zlib_ep-install")
+  set(ZLIB_HOME "${ZLIB_PREFIX}")
+  set(ZLIB_INCLUDE_DIR "${ZLIB_PREFIX}/include")
+  if (MSVC)
+    if (${UPPERCASE_BUILD_TYPE} STREQUAL "DEBUG")
+      set(ZLIB_STATIC_LIB_NAME zlibstaticd.lib)
+    else()
+      set(ZLIB_STATIC_LIB_NAME zlibstatic.lib)
+    endif()
+  else()
+    set(ZLIB_STATIC_LIB_NAME libz.a)
+  endif()
+  set(ZLIB_STATIC_LIB "${ZLIB_PREFIX}/lib/${ZLIB_STATIC_LIB_NAME}")
+  set(ZLIB_CMAKE_ARGS -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}
+    -DCMAKE_INSTALL_PREFIX=${ZLIB_PREFIX}
+    -DCMAKE_C_FLAGS=${EP_C_FLAGS}
+    -DBUILD_SHARED_LIBS=OFF)
+  ExternalProject_Add(zlib_ep
+    URL "http://zlib.net/fossils/zlib-1.2.8.tar.gz"
+    BUILD_BYPRODUCTS "${ZLIB_STATIC_LIB}"
+    ${ZLIB_BUILD_BYPRODUCTS}
+    ${EP_LOG_OPTIONS}
+    CMAKE_ARGS ${ZLIB_CMAKE_ARGS})
+
+  set(THRIFT_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/thrift_ep/src/thrift_ep-install")
+  set(THRIFT_HOME "${THRIFT_PREFIX}")
+  set(THRIFT_INCLUDE_DIR "${THRIFT_PREFIX}/include")
+  set(THRIFT_COMPILER "${THRIFT_PREFIX}/bin/thrift")
+  set(THRIFT_CMAKE_ARGS "-DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}"
+                        "-DCMAKE_CXX_FLAGS=${EP_CXX_FLAGS}"
+                        "-DCMAKE_C_FLAGS=${EP_C_FLAGS}"
+                        "-DCMAKE_INSTALL_PREFIX=${THRIFT_PREFIX}"
+                        "-DCMAKE_INSTALL_RPATH=${THRIFT_PREFIX}/lib"
+                        "-DBUILD_SHARED_LIBS=OFF"
+                        "-DBUILD_TESTING=OFF"
+                        "-DBUILD_EXAMPLES=OFF"
+                        "-DBUILD_TUTORIALS=OFF"
+                        "-DWITH_QT4=OFF"
+                        "-DWITH_C_GLIB=OFF"
+                        "-DWITH_JAVA=OFF"
+                        "-DWITH_PYTHON=OFF"
+                        "-DWITH_HASKELL=OFF"
+                        "-DWITH_CPP=ON"
+                        "-DWITH_STATIC_LIB=ON"
+                        "-DWITH_LIBEVENT=OFF"
+                        )
+
+  # Thrift also uses boost. Forward important boost settings if there were ones passed.
+  if (DEFINED BOOST_ROOT)
+    set(THRIFT_CMAKE_ARGS ${THRIFT_CMAKE_ARGS} "-DBOOST_ROOT=${BOOST_ROOT}")
+  endif()
+  if (DEFINED Boost_NAMESPACE)
+    set(THRIFT_CMAKE_ARGS ${THRIFT_CMAKE_ARGS} "-DBoost_NAMESPACE=${Boost_NAMESPACE}")
+  endif()
+
+  set(THRIFT_STATIC_LIB_NAME "${CMAKE_STATIC_LIBRARY_PREFIX}thrift")
+  if (MSVC)
+    if (ARROW_USE_STATIC_CRT)
+      set(THRIFT_STATIC_LIB_NAME "${THRIFT_STATIC_LIB_NAME}mt")
+      set(THRIFT_CMAKE_ARGS ${THRIFT_CMAKE_ARGS} "-DWITH_MT=ON")
+    else()
+      set(THRIFT_STATIC_LIB_NAME "${THRIFT_STATIC_LIB_NAME}md")
+      set(THRIFT_CMAKE_ARGS ${THRIFT_CMAKE_ARGS} "-DWITH_MT=OFF")
+    endif()
+  endif()
+  if (${UPPERCASE_BUILD_TYPE} STREQUAL "DEBUG")
+    set(THRIFT_STATIC_LIB_NAME "${THRIFT_STATIC_LIB_NAME}d")
+  endif()
+  set(THRIFT_STATIC_LIB "${THRIFT_PREFIX}/lib/${THRIFT_STATIC_LIB_NAME}${CMAKE_STATIC_LIBRARY_SUFFIX}")
+
+  if (MSVC)
+    set(WINFLEXBISON_VERSION 2.4.9)
+    set(WINFLEXBISON_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/winflexbison_ep/src/winflexbison_ep-install")
+    ExternalProject_Add(winflexbison_ep
+      URL https://github.com/lexxmark/winflexbison/releases/download/v.${WINFLEXBISON_VERSION}/win_flex_bison-${WINFLEXBISON_VERSION}.zip
+      URL_HASH MD5=a2e979ea9928fbf8567e995e9c0df765
+      SOURCE_DIR ${WINFLEXBISON_PREFIX}
+      CONFIGURE_COMMAND ""
+      BUILD_COMMAND ""
+      INSTALL_COMMAND ""
+      ${EP_LOG_OPTIONS})
+    set(THRIFT_DEPENDENCIES ${THRIFT_DEPENDENCIES} winflexbison_ep)
+
+    set(THRIFT_CMAKE_ARGS "-DFLEX_EXECUTABLE=${WINFLEXBISON_PREFIX}/win_flex.exe"
+                          "-DBISON_EXECUTABLE=${WINFLEXBISON_PREFIX}/win_bison.exe"
+                          "-DZLIB_INCLUDE_DIR=${ZLIB_INCLUDE_DIR}"
+                          "-DZLIB_LIBRARY=${ZLIB_STATIC_LIB}"
+                          "-DWITH_SHARED_LIB=OFF"
+                          "-DWITH_PLUGIN=OFF"
+                          ${THRIFT_CMAKE_ARGS})
+    set(THRIFT_DEPENDENCIES ${THRIFT_DEPENDENCIES} zlib_ep)
+  elseif (APPLE)
+    if (DEFINED BISON_EXECUTABLE)
+      set(THRIFT_CMAKE_ARGS "-DBISON_EXECUTABLE=${BISON_EXECUTABLE}"
+                            ${THRIFT_CMAKE_ARGS})
+    endif()
+  endif()
+
+  ExternalProject_Add(thrift_ep
+    URL ${THRIFT_SOURCE_URL}
+    BUILD_BYPRODUCTS "${THRIFT_STATIC_LIB}" "${THRIFT_COMPILER}"
+    CMAKE_ARGS ${THRIFT_CMAKE_ARGS}
+    DEPENDS ${THRIFT_DEPENDENCIES}
+    ${EP_LOG_OPTIONS})
+
+  set(THRIFT_VENDORED 1)
+else()
+  set(THRIFT_VENDORED 0)
+endif()
+
+include_directories(SYSTEM ${THRIFT_INCLUDE_DIR} ${THRIFT_INCLUDE_DIR}/thrift)
+message(STATUS "Thrift include dir: ${THRIFT_INCLUDE_DIR}")
+message(STATUS "Thrift static library: ${THRIFT_STATIC_LIB}")
+message(STATUS "Thrift compiler: ${THRIFT_COMPILER}")
+message(STATUS "Thrift version: ${THRIFT_VERSION}")
+add_library(thriftstatic STATIC IMPORTED)
+set_target_properties(thriftstatic PROPERTIES IMPORTED_LOCATION ${THRIFT_STATIC_LIB})
+
+if (THRIFT_VENDORED)
+  add_dependencies(thriftstatic thrift_ep)
+endif()
+
+endif()  # ARROW_HIVESERVER2
diff --git a/cpp/src/arrow/dbi/README.md b/cpp/src/arrow/dbi/README.md
new file mode 100644
index 0000000..d73666c
--- /dev/null
+++ b/cpp/src/arrow/dbi/README.md
@@ -0,0 +1,24 @@
+<!---
+  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.
+-->
+
+# Arrow Database Interfaces
+
+## HiveServer2
+
+For Apache Hive and Apache Impala. See `hiveserver2/` directory
diff --git a/cpp/src/arrow/dbi/hiveserver2/CMakeLists.txt b/cpp/src/arrow/dbi/hiveserver2/CMakeLists.txt
new file mode 100644
index 0000000..60d88da
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/CMakeLists.txt
@@ -0,0 +1,110 @@
+# 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.
+
+# Headers: top level
+install(FILES
+  api.h
+  columnar-row-set.h
+  operation.h
+  service.h
+  session.h
+  types.h
+  util.h
+  DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/arrow/dbi/hiveserver2")
+
+set(ARROW_HIVESERVER2_SRCS
+  columnar-row-set.cc
+  service.cc
+  session.cc
+  operation.cc
+  sample-usage.cc
+  thrift-internal.cc
+  types.cc
+  util.cc
+)
+
+add_subdirectory(thrift)
+
+set(HIVESERVER2_THRIFT_SRC
+  ErrorCodes_constants.cpp
+  ErrorCodes_types.cpp
+  ImpalaService.cpp
+  ImpalaService_constants.cpp
+  ImpalaService_types.cpp
+  ImpalaHiveServer2Service.cpp
+  beeswax_constants.cpp
+  beeswax_types.cpp
+  BeeswaxService.cpp
+  TCLIService.cpp
+  TCLIService_constants.cpp
+  TCLIService_types.cpp
+  ExecStats_constants.cpp
+  ExecStats_types.cpp
+  hive_metastore_constants.cpp
+  hive_metastore_types.cpp
+  Status_constants.cpp
+  Status_types.cpp
+  Types_constants.cpp
+  Types_types.cpp
+)
+
+set_source_files_properties(${HIVESERVER2_THRIFT_SRC} PROPERTIES
+  COMPILE_FLAGS "-Wno-unused-variable -Wno-shadow-field"
+  GENERATED TRUE)
+
+# keep everything in one library, the object files reference
+# each other
+add_library(arrow_hiveserver2_thrift STATIC ${HIVESERVER2_THRIFT_SRC})
+
+# Setting these files as code-generated lets make clean and incremental builds work
+# correctly
+
+# TODO(wesm): Something is broken with the dependency chain with
+# ImpalaService.cpp and others. Couldn't figure out what is different between
+# this setup and Impala.
+
+add_dependencies(arrow_hiveserver2_thrift hs2-thrift-cpp)
+
+set_target_properties(arrow_hiveserver2_thrift
+  PROPERTIES
+  LIBRARY_OUTPUT_DIRECTORY "${BUILD_OUTPUT_ROOT_DIRECTORY}")
+
+ADD_ARROW_LIB(arrow_hiveserver2
+  SOURCES ${ARROW_HIVESERVER2_SRCS}
+  DEPENDENCIES arrow_hiveserver2_thrift
+  SHARED_LINK_FLAGS ""
+  SHARED_LINK_LIBS ${ARROW_PYTHON_SHARED_LINK_LIBS}
+)
+
+set_property(SOURCE ${ARROW_HIVESERVER2_SRCS}
+  APPEND_STRING PROPERTY COMPILE_FLAGS
+  " -Wno-shadow-field")
+
+set(ARROW_HIVESERVER2_TEST_LINK_LIBS
+  ${ARROW_TEST_LINK_LIBS}
+  arrow_hiveserver2_static
+  arrow_hiveserver2_thrift
+  thriftstatic)
+
+ADD_ARROW_TEST(hiveserver2-test
+  STATIC_LINK_LIBS "${ARROW_HIVESERVER2_TEST_LINK_LIBS}"
+  LABELS "hiveserver2"
+)
+
+set_property(TARGET hiveserver2-test
+  APPEND_STRING PROPERTY COMPILE_FLAGS
+  " -Wno-shadow-field")
diff --git a/cpp/src/arrow/dbi/hiveserver2/api.h b/cpp/src/arrow/dbi/hiveserver2/api.h
new file mode 100644
index 0000000..6ac849e
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/api.h
@@ -0,0 +1,27 @@
+// 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.
+
+#pragma once
+
+#include "arrow/dbi/hiveserver2/columnar-row-set.h"
+#include "arrow/dbi/hiveserver2/operation.h"
+#include "arrow/dbi/hiveserver2/service.h"
+#include "arrow/dbi/hiveserver2/session.h"
+#include "arrow/dbi/hiveserver2/types.h"
+#include "arrow/dbi/hiveserver2/util.h"
+
+#include "arrow/status.h"
diff --git a/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.cc b/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.cc
new file mode 100644
index 0000000..c3d654e
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.cc
@@ -0,0 +1,100 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/columnar-row-set.h"
+
+#include <string>
+#include <vector>
+
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/util/logging.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+
+namespace arrow {
+namespace hiveserver2 {
+
+Column::Column(const std::string* nulls) {
+  DCHECK(nulls);
+  nulls_ = reinterpret_cast<const uint8_t*>(nulls->c_str());
+  nulls_size_ = static_cast<int64_t>(nulls->size());
+}
+
+ColumnarRowSet::ColumnarRowSet(ColumnarRowSetImpl* impl) : impl_(impl) {}
+
+ColumnarRowSet::~ColumnarRowSet() = default;
+
+template <typename T>
+struct type_helpers {};
+
+#define VALUE_GETTER(COLUMN_TYPE, VALUE_TYPE, ATTR_NAME)                       \
+  template <>                                                                  \
+  struct type_helpers<COLUMN_TYPE> {                                           \
+    static const std::vector<VALUE_TYPE>* GetValues(const hs2::TColumn& col) { \
+      return &col.ATTR_NAME.values;                                            \
+    }                                                                          \
+                                                                               \
+    static const std::string* GetNulls(const hs2::TColumn& col) {              \
+      return &col.ATTR_NAME.nulls;                                             \
+    }                                                                          \
+  };
+
+VALUE_GETTER(BoolColumn, bool, boolVal);
+VALUE_GETTER(ByteColumn, int8_t, byteVal);
+VALUE_GETTER(Int16Column, int16_t, i16Val);
+VALUE_GETTER(Int32Column, int32_t, i32Val);
+VALUE_GETTER(Int64Column, int64_t, i64Val);
+VALUE_GETTER(DoubleColumn, double, doubleVal);
+VALUE_GETTER(StringColumn, std::string, stringVal);
+
+#undef VALUE_GETTER
+
+template <typename T>
+std::unique_ptr<T> ColumnarRowSet::GetCol(int i) const {
+  using helper = type_helpers<T>;
+
+  DCHECK_LT(i, static_cast<int>(impl_->resp.results.columns.size()));
+
+  const hs2::TColumn& col = impl_->resp.results.columns[i];
+  return std::unique_ptr<T>(new T(helper::GetNulls(col), helper::GetValues(col)));
+}
+
+#define TYPED_GETTER(FUNC_NAME, TYPE)                            \
+  std::unique_ptr<TYPE> ColumnarRowSet::FUNC_NAME(int i) const { \
+    return GetCol<TYPE>(i);                                      \
+  }                                                              \
+  template std::unique_ptr<TYPE> ColumnarRowSet::GetCol<TYPE>(int i) const;
+
+TYPED_GETTER(GetBoolCol, BoolColumn);
+TYPED_GETTER(GetByteCol, ByteColumn);
+TYPED_GETTER(GetInt16Col, Int16Column);
+TYPED_GETTER(GetInt32Col, Int32Column);
+TYPED_GETTER(GetInt64Col, Int64Column);
+TYPED_GETTER(GetDoubleCol, DoubleColumn);
+TYPED_GETTER(GetStringCol, StringColumn);
+
+#undef TYPED_GETTER
+
+// BinaryColumn is an alias for StringColumn
+std::unique_ptr<BinaryColumn> ColumnarRowSet::GetBinaryCol(int i) const {
+  return GetCol<BinaryColumn>(i);
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.h b/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.h
new file mode 100644
index 0000000..a62c738
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/columnar-row-set.h
@@ -0,0 +1,155 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace hiveserver2 {
+
+// The Column class is used to access data that was fetched in columnar format.
+// The contents of the data can be accessed through the data() fn, which returns
+// a ptr to a vector containing the contents of this column in the fetched
+// results, avoiding copies. This vector will be of size length().
+//
+// If any of the values are null, they will be represented in the data vector as
+// default values, i.e. 0 for numeric types. The nulls() fn returns a ptr to a
+// bit array representing which values are null, and the IsNull() fn is provided
+// for convenience when working with this bit array. The user should check
+// IsNull() to distinguish between actual instances of the default values and nulls.
+//
+// A Column object is returned from a ColumnarRowSet and is only valid as long
+// as that ColumnarRowSet still exists.
+//
+// Example:
+// unique_ptr<Int32Column> col = columnar_row_set->GetInt32Col();
+// for (int i = 0; i < col->length(); i++) {
+//   if (col->IsNull(i)) {
+//     cout << "NULL\n";
+//   } else {
+//     cout << col->data()[i] << "\n";
+//   }
+// }
+class ARROW_EXPORT Column {
+ public:
+  virtual ~Column() {}
+
+  virtual int64_t length() const = 0;
+
+  const uint8_t* nulls() const { return nulls_; }
+  int64_t nulls_size() const { return nulls_size_; }
+
+  // Returns true iff the value for the i-th row within this set of data for this
+  // column is null.
+  bool IsNull(int64_t i) const { return (nulls_[i / 8] & (1 << (i % 8))) != 0; }
+
+ protected:
+  explicit Column(const std::string* nulls);
+
+  // The memory for these ptrs is owned by the ColumnarRowSet that
+  // created this Column.
+  //
+  // Due to the issue described in HUE-2722, the null bitmap may have fewer
+  // bytes than expected for some versions of Hive, so we retain the ability to
+  // check the buffer size in case this happens.
+  const uint8_t* nulls_;
+  int64_t nulls_size_;
+};
+
+template <class T>
+class ARROW_EXPORT TypedColumn : public Column {
+ public:
+  const std::vector<T>& data() const { return *data_; }
+  int64_t length() const { return data().size(); }
+
+  // Returns the value for the i-th row within this set of data for this column.
+  const T& GetData(int64_t i) const { return data()[i]; }
+
+ private:
+  // For access to the c'tor.
+  friend class ColumnarRowSet;
+
+  TypedColumn(const std::string* nulls, const std::vector<T>* data)
+      : Column(nulls), data_(data) {}
+
+  const std::vector<T>* data_;
+};
+
+typedef TypedColumn<bool> BoolColumn;
+typedef TypedColumn<int8_t> ByteColumn;
+typedef TypedColumn<int16_t> Int16Column;
+typedef TypedColumn<int32_t> Int32Column;
+typedef TypedColumn<int64_t> Int64Column;
+typedef TypedColumn<double> DoubleColumn;
+typedef TypedColumn<std::string> StringColumn;
+typedef TypedColumn<std::string> BinaryColumn;
+
+// A ColumnarRowSet represents the full results returned by a call to
+// Operation::Fetch() when a columnar format is being used.
+//
+// ColumnarRowSet provides access to specific columns by their type and index in
+// the results. All Column objects returned from a given ColumnarRowSet will have
+// the same length(). A Column object returned by a ColumnarRowSet is only valid
+// as long as the ColumnarRowSet still exists.
+//
+// Example:
+// unique_ptr<Operation> op;
+// session->ExecuteStatement("select int_col, string_col from tbl", &op);
+// unique_ptr<ColumnarRowSet> columnar_row_set;
+// if (op->Fetch(&columnar_row_set).ok()) {
+//   unique_ptr<Int32Column> int32_col = columnar_row_set->GetInt32Col(0);
+//   unique_ptr<StringColumn> string_col = columnar_row_set->GetStringCol(1);
+// }
+class ARROW_EXPORT ColumnarRowSet {
+ public:
+  ~ColumnarRowSet();
+
+  std::unique_ptr<BoolColumn> GetBoolCol(int i) const;
+  std::unique_ptr<ByteColumn> GetByteCol(int i) const;
+  std::unique_ptr<Int16Column> GetInt16Col(int i) const;
+  std::unique_ptr<Int32Column> GetInt32Col(int i) const;
+  std::unique_ptr<Int64Column> GetInt64Col(int i) const;
+  std::unique_ptr<DoubleColumn> GetDoubleCol(int i) const;
+  std::unique_ptr<StringColumn> GetStringCol(int i) const;
+  std::unique_ptr<BinaryColumn> GetBinaryCol(int i) const;
+
+  template <typename T>
+  std::unique_ptr<T> GetCol(int i) const;
+
+ private:
+  // Hides Thrift objects from the header.
+  struct ColumnarRowSetImpl;
+
+  ARROW_DISALLOW_COPY_AND_ASSIGN(ColumnarRowSet);
+
+  // For access to the c'tor.
+  friend class Operation;
+
+  explicit ColumnarRowSet(ColumnarRowSetImpl* impl);
+
+  std::unique_ptr<ColumnarRowSetImpl> impl_;
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc b/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc
new file mode 100644
index 0000000..7022ff0
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc
@@ -0,0 +1,462 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/operation.h"
+
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/dbi/hiveserver2/service.h"
+#include "arrow/dbi/hiveserver2/session.h"
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/status.h"
+#include "arrow/test-util.h"
+
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+namespace arrow {
+namespace hiveserver2 {
+
+static std::string GetTestHost() {
+  const char* host = std::getenv("ARROW_HIVESERVER2_TEST_HOST");
+  return host == nullptr ? "localhost" : std::string(host);
+}
+
+// Convenience functions for finding a row of values given several columns.
+template <typename VType, typename CType>
+bool FindRow(VType value, CType* column) {
+  for (int i = 0; i < column->length(); ++i) {
+    if (column->data()[i] == value) {
+      return true;
+    }
+  }
+  return false;
+}
+
+template <typename V1Type, typename V2Type, typename C1Type, typename C2Type>
+bool FindRow(V1Type value1, V2Type value2, C1Type* column1, C2Type* column2) {
+  EXPECT_EQ(column1->length(), column2->length());
+  for (int i = 0; i < column1->length(); ++i) {
+    if (column1->data()[i] == value1 && column2->data()[i] == value2) {
+      return true;
+    }
+  }
+  return false;
+}
+
+template <typename V1Type, typename V2Type, typename V3Type, typename C1Type,
+          typename C2Type, typename C3Type>
+bool FindRow(V1Type value1, V2Type value2, V3Type value3, C1Type* column1,
+             C2Type* column2, C3Type column3) {
+  EXPECT_EQ(column1->length(), column2->length());
+  EXPECT_EQ(column1->length(), column3->length());
+  for (int i = 0; i < column1->length(); ++i) {
+    if (column1->data()[i] == value1 && column2->data()[i] == value2 &&
+        column3->data()[i] == value3) {
+      return true;
+    }
+  }
+  return false;
+}
+
+// Waits for this operation to reach the given state, sleeping for sleep microseconds
+// between checks, and failing after max_retries checks.
+Status Wait(const std::unique_ptr<Operation>& op,
+            Operation::State state = Operation::State::FINISHED, int sleep_us = 10000,
+            int max_retries = 100) {
+  int retries = 0;
+  Operation::State op_state;
+  RETURN_NOT_OK(op->GetState(&op_state));
+  while (op_state != state && retries < max_retries) {
+    usleep(sleep_us);
+    RETURN_NOT_OK(op->GetState(&op_state));
+    ++retries;
+  }
+
+  if (op_state == state) {
+    return Status::OK();
+  } else {
+    std::stringstream ss;
+    ss << "Failed to reach state '" << OperationStateToString(state) << "' after "
+       << retries << " retries.";
+    return Status::IOError(ss.str());
+  }
+}
+
+// Creates a service, session, and database for use in tests.
+class HS2ClientTest : public ::testing::Test {
+ protected:
+  virtual void SetUp() {
+    hostname_ = GetTestHost();
+
+    int conn_timeout = 0;
+    ProtocolVersion protocol_version = ProtocolVersion::PROTOCOL_V7;
+    ASSERT_OK(
+        Service::Connect(hostname_, port, conn_timeout, protocol_version, &service_));
+
+    std::string user = "user";
+    HS2ClientConfig config;
+    ASSERT_OK(service_->OpenSession(user, config, &session_));
+
+    std::unique_ptr<Operation> drop_db_op;
+    ASSERT_OK(session_->ExecuteStatement(
+        "drop database if exists " + TEST_DB + " cascade", &drop_db_op));
+    ASSERT_OK(drop_db_op->Close());
+
+    std::unique_ptr<Operation> create_db_op;
+    ASSERT_OK(session_->ExecuteStatement("create database " + TEST_DB, &create_db_op));
+    ASSERT_OK(create_db_op->Close());
+
+    std::unique_ptr<Operation> use_db_op;
+    ASSERT_OK(session_->ExecuteStatement("use " + TEST_DB, &use_db_op));
+    ASSERT_OK(use_db_op->Close());
+  }
+
+  virtual void TearDown() {
+    std::unique_ptr<Operation> use_db_op;
+    if (session_) {
+      // We were able to create a session and service
+      ASSERT_OK(session_->ExecuteStatement("use default", &use_db_op));
+      ASSERT_OK(use_db_op->Close());
+
+      std::unique_ptr<Operation> drop_db_op;
+      ASSERT_OK(session_->ExecuteStatement("drop database " + TEST_DB + " cascade",
+                                           &drop_db_op));
+      ASSERT_OK(drop_db_op->Close());
+
+      ASSERT_OK(session_->Close());
+      ASSERT_OK(service_->Close());
+    }
+  }
+
+  void CreateTestTable() {
+    std::unique_ptr<Operation> create_table_op;
+    ASSERT_OK(session_->ExecuteStatement(
+        "create table " + TEST_TBL + " (" + TEST_COL1 + " int, " + TEST_COL2 + " string)",
+        &create_table_op));
+    ASSERT_OK(create_table_op->Close());
+  }
+
+  void InsertIntoTestTable(std::vector<int> int_col_data,
+                           std::vector<std::string> string_col_data) {
+    ASSERT_EQ(int_col_data.size(), string_col_data.size());
+
+    std::stringstream query;
+    query << "insert into " << TEST_TBL << " VALUES ";
+    for (size_t i = 0; i < int_col_data.size(); i++) {
+      if (int_col_data[i] == NULL_INT_VALUE) {
+        query << " (NULL, ";
+      } else {
+        query << " (" << int_col_data[i] << ", ";
+      }
+
+      if (string_col_data[i] == "NULL") {
+        query << "NULL)";
+      } else {
+        query << "'" << string_col_data[i] << "')";
+      }
+
+      if (i != int_col_data.size() - 1) {
+        query << ", ";
+      }
+    }
+
+    std::unique_ptr<Operation> insert_op;
+    ASSERT_OK(session_->ExecuteStatement(query.str(), &insert_op));
+    ASSERT_OK(Wait(insert_op));
+    Operation::State insert_op_state;
+    ASSERT_OK(insert_op->GetState(&insert_op_state));
+    ASSERT_EQ(insert_op_state, Operation::State::FINISHED);
+    ASSERT_OK(insert_op->Close());
+  }
+  std::string hostname_;
+
+  int port = 21050;
+
+  const std::string TEST_DB = "hs2client_test_db";
+  const std::string TEST_TBL = "hs2client_test_table";
+  const std::string TEST_COL1 = "int_col";
+  const std::string TEST_COL2 = "string_col";
+
+  const int NULL_INT_VALUE = -1;
+
+  std::unique_ptr<Service> service_;
+  std::unique_ptr<Session> session_;
+};
+
+class OperationTest : public HS2ClientTest {};
+
+TEST_F(OperationTest, TestFetch) {
+  CreateTestTable();
+  InsertIntoTestTable(vector<int>({1, 2, 3, 4}), vector<string>({"a", "b", "c", "d"}));
+
+  unique_ptr<Operation> select_op;
+  ASSERT_OK(session_->ExecuteStatement("select * from " + TEST_TBL + " order by int_col",
+                                       &select_op));
+
+  unique_ptr<ColumnarRowSet> results;
+  bool has_more_rows = false;
+  // Impala only supports NEXT and FIRST.
+  ASSERT_RAISES(IOError,
+                select_op->Fetch(2, FetchOrientation::LAST, &results, &has_more_rows));
+
+  // Fetch the results in two batches by passing max_rows to Fetch.
+  ASSERT_OK(select_op->Fetch(2, FetchOrientation::NEXT, &results, &has_more_rows));
+  ASSERT_OK(Wait(select_op));
+  ASSERT_TRUE(select_op->HasResultSet());
+  unique_ptr<Int32Column> int_col = results->GetInt32Col(0);
+  unique_ptr<StringColumn> string_col = results->GetStringCol(1);
+  ASSERT_EQ(int_col->data(), vector<int>({1, 2}));
+  ASSERT_EQ(string_col->data(), vector<string>({"a", "b"}));
+  ASSERT_TRUE(has_more_rows);
+
+  ASSERT_OK(select_op->Fetch(2, FetchOrientation::NEXT, &results, &has_more_rows));
+  int_col = results->GetInt32Col(0);
+  string_col = results->GetStringCol(1);
+  ASSERT_EQ(int_col->data(), vector<int>({3, 4}));
+  ASSERT_EQ(string_col->data(), vector<string>({"c", "d"}));
+
+  ASSERT_OK(select_op->Fetch(2, FetchOrientation::NEXT, &results, &has_more_rows));
+  int_col = results->GetInt32Col(0);
+  string_col = results->GetStringCol(1);
+  ASSERT_EQ(int_col->length(), 0);
+  ASSERT_EQ(string_col->length(), 0);
+  ASSERT_FALSE(has_more_rows);
+
+  ASSERT_OK(select_op->Fetch(2, FetchOrientation::NEXT, &results, &has_more_rows));
+  int_col = results->GetInt32Col(0);
+  string_col = results->GetStringCol(1);
+  ASSERT_EQ(int_col->length(), 0);
+  ASSERT_EQ(string_col->length(), 0);
+  ASSERT_FALSE(has_more_rows);
+
+  ASSERT_OK(select_op->Close());
+}
+
+TEST_F(OperationTest, TestIsNull) {
+  CreateTestTable();
+  // Insert some NULLs and ensure Column::IsNull() is correct.
+  InsertIntoTestTable(vector<int>({1, 2, 3, 4, 5, NULL_INT_VALUE}),
+                      vector<string>({"a", "b", "NULL", "d", "NULL", "f"}));
+
+  unique_ptr<Operation> select_nulls_op;
+  ASSERT_OK(session_->ExecuteStatement("select * from " + TEST_TBL + " order by int_col",
+                                       &select_nulls_op));
+
+  unique_ptr<ColumnarRowSet> nulls_results;
+  bool has_more_rows = false;
+  ASSERT_OK(select_nulls_op->Fetch(&nulls_results, &has_more_rows));
+  unique_ptr<Int32Column> int_col = nulls_results->GetInt32Col(0);
+  unique_ptr<StringColumn> string_col = nulls_results->GetStringCol(1);
+  ASSERT_EQ(int_col->length(), 6);
+  ASSERT_EQ(int_col->length(), string_col->length());
+
+  bool int_nulls[] = {false, false, false, false, false, true};
+  for (int i = 0; i < int_col->length(); i++) {
+    ASSERT_EQ(int_col->IsNull(i), int_nulls[i]);
+  }
+  bool string_nulls[] = {false, false, true, false, true, false};
+  for (int i = 0; i < string_col->length(); i++) {
+    ASSERT_EQ(string_col->IsNull(i), string_nulls[i]);
+  }
+
+  ASSERT_OK(select_nulls_op->Close());
+}
+
+TEST_F(OperationTest, TestCancel) {
+  CreateTestTable();
+  InsertIntoTestTable(vector<int>({1, 2, 3, 4}), vector<string>({"a", "b", "c", "d"}));
+
+  unique_ptr<Operation> op;
+  ASSERT_OK(session_->ExecuteStatement("select count(*) from " + TEST_TBL, &op));
+  ASSERT_OK(op->Cancel());
+  // Impala currently returns ERROR and not CANCELED for canceled queries
+  // due to the use of beeswax states, which don't support a canceled state.
+  ASSERT_OK(Wait(op, Operation::State::ERROR));
+
+  string profile;
+  ASSERT_OK(op->GetProfile(&profile));
+  ASSERT_TRUE(profile.find("Cancelled") != string::npos);
+
+  ASSERT_OK(op->Close());
+}
+
+TEST_F(OperationTest, TestGetLog) {
+  CreateTestTable();
+
+  unique_ptr<Operation> op;
+  ASSERT_OK(session_->ExecuteStatement("select count(*) from " + TEST_TBL, &op));
+  string log;
+  ASSERT_OK(op->GetLog(&log));
+  ASSERT_NE(log, "");
+
+  ASSERT_OK(op->Close());
+}
+
+TEST_F(OperationTest, TestGetResultSetMetadata) {
+  const string TEST_COL1 = "int_col";
+  const string TEST_COL2 = "varchar_col";
+  const int MAX_LENGTH = 10;
+  const string TEST_COL3 = "decimal_cal";
+  const int PRECISION = 5;
+  const int SCALE = 3;
+  std::stringstream create_query;
+  create_query << "create table " << TEST_TBL << " (" << TEST_COL1 << " int, "
+               << TEST_COL2 << " varchar(" << MAX_LENGTH << "), " << TEST_COL3
+               << " decimal(" << PRECISION << ", " << SCALE << "))";
+  unique_ptr<Operation> create_table_op;
+  ASSERT_OK(session_->ExecuteStatement(create_query.str(), &create_table_op));
+  ASSERT_OK(create_table_op->Close());
+
+  // Perform a select, and check that we get the right metadata back.
+  unique_ptr<Operation> select_op;
+  ASSERT_OK(session_->ExecuteStatement("select * from " + TEST_TBL, &select_op));
+  vector<ColumnDesc> column_descs;
+  ASSERT_OK(select_op->GetResultSetMetadata(&column_descs));
+  ASSERT_EQ(column_descs.size(), 3);
+
+  ASSERT_EQ(column_descs[0].column_name(), TEST_COL1);
+  ASSERT_EQ(column_descs[0].type()->ToString(), "INT");
+  ASSERT_EQ(column_descs[0].type()->type_id(), ColumnType::TypeId::INT);
+  ASSERT_EQ(column_descs[0].position(), 0);
+
+  ASSERT_EQ(column_descs[1].column_name(), TEST_COL2);
+  ASSERT_EQ(column_descs[1].type()->ToString(), "VARCHAR");
+  ASSERT_EQ(column_descs[1].type()->type_id(), ColumnType::TypeId::VARCHAR);
+  ASSERT_EQ(column_descs[1].position(), 1);
+  ASSERT_EQ(column_descs[1].GetCharacterType()->max_length(), MAX_LENGTH);
+
+  ASSERT_EQ(column_descs[2].column_name(), TEST_COL3);
+  ASSERT_EQ(column_descs[2].type()->ToString(), "DECIMAL");
+  ASSERT_EQ(column_descs[2].type()->type_id(), ColumnType::TypeId::DECIMAL);
+  ASSERT_EQ(column_descs[2].position(), 2);
+  ASSERT_EQ(column_descs[2].GetDecimalType()->precision(), PRECISION);
+  ASSERT_EQ(column_descs[2].GetDecimalType()->scale(), SCALE);
+
+  ASSERT_OK(select_op->Close());
+
+  // Insert ops don't have result sets.
+  std::stringstream insert_query;
+  insert_query << "insert into " << TEST_TBL << " VALUES (1, cast('a' as varchar("
+               << MAX_LENGTH << ")), cast(1 as decimal(" << PRECISION << ", " << SCALE
+               << ")))";
+  unique_ptr<Operation> insert_op;
+  ASSERT_OK(session_->ExecuteStatement(insert_query.str(), &insert_op));
+  vector<ColumnDesc> insert_column_descs;
+  ASSERT_OK(insert_op->GetResultSetMetadata(&insert_column_descs));
+  ASSERT_EQ(insert_column_descs.size(), 0);
+  ASSERT_OK(insert_op->Close());
+}
+
+class SessionTest : public HS2ClientTest {};
+
+TEST_F(SessionTest, TestSessionConfig) {
+  // Create a table in TEST_DB.
+  const string& TEST_TBL = "hs2client_test_table";
+  unique_ptr<Operation> create_table_op;
+  ASSERT_OK(session_->ExecuteStatement(
+      "create table " + TEST_TBL + " (int_col int, string_col string)",
+      &create_table_op));
+  ASSERT_OK(create_table_op->Close());
+
+  // Start a new session with the use:database session option.
+  string user = "user";
+  HS2ClientConfig config_use;
+  config_use.SetOption("use:database", TEST_DB);
+  unique_ptr<Session> session_ok;
+  ASSERT_OK(service_->OpenSession(user, config_use, &session_ok));
+
+  // Ensure the use:database worked and we can access the table.
+  unique_ptr<Operation> select_op;
+  ASSERT_OK(session_ok->ExecuteStatement("select * from " + TEST_TBL, &select_op));
+  ASSERT_OK(select_op->Close());
+  ASSERT_OK(session_ok->Close());
+
+  // Start another session without use:database.
+  HS2ClientConfig config_no_use;
+  unique_ptr<Session> session_error;
+  ASSERT_OK(service_->OpenSession(user, config_no_use, &session_error));
+
+  // Ensure the we can't access the table.
+  unique_ptr<Operation> select_op_error;
+  ASSERT_RAISES(IOError, session_error->ExecuteStatement("select * from " + TEST_TBL,
+                                                         &select_op_error));
+  ASSERT_OK(session_error->Close());
+}
+
+TEST(ServiceTest, TestConnect) {
+  // Open a connection.
+  string host = GetTestHost();
+  int port = 21050;
+  int conn_timeout = 0;
+  ProtocolVersion protocol_version = ProtocolVersion::PROTOCOL_V7;
+  unique_ptr<Service> service;
+  ASSERT_OK(Service::Connect(host, port, conn_timeout, protocol_version, &service));
+  ASSERT_TRUE(service->IsConnected());
+
+  // Check that we can start a session.
+  string user = "user";
+  HS2ClientConfig config;
+  unique_ptr<Session> session1;
+  ASSERT_OK(service->OpenSession(user, config, &session1));
+  ASSERT_OK(session1->Close());
+
+  // Close the service. We should not be able to open a session.
+  ASSERT_OK(service->Close());
+  ASSERT_FALSE(service->IsConnected());
+  ASSERT_OK(service->Close());
+  unique_ptr<Session> session3;
+  ASSERT_RAISES(IOError, service->OpenSession(user, config, &session3));
+  ASSERT_OK(session3->Close());
+
+  // We should be able to call Close again without errors.
+  ASSERT_OK(service->Close());
+  ASSERT_FALSE(service->IsConnected());
+}
+
+TEST(ServiceTest, TestFailedConnect) {
+  string host = GetTestHost();
+  int port = 21050;
+
+  // Set 100ms timeout so these return quickly
+  int conn_timeout = 100;
+
+  ProtocolVersion protocol_version = ProtocolVersion::PROTOCOL_V7;
+  unique_ptr<Service> service;
+
+  string invalid_host = "does_not_exist";
+  ASSERT_RAISES(IOError, Service::Connect(invalid_host, port, conn_timeout,
+                                          protocol_version, &service));
+
+  int invalid_port = -1;
+  ASSERT_RAISES(IOError, Service::Connect(host, invalid_port, conn_timeout,
+                                          protocol_version, &service));
+
+  ProtocolVersion invalid_protocol_version = ProtocolVersion::PROTOCOL_V2;
+  ASSERT_RAISES(NotImplemented, Service::Connect(host, port, conn_timeout,
+                                                 invalid_protocol_version, &service));
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/operation.cc b/cpp/src/arrow/dbi/hiveserver2/operation.cc
new file mode 100644
index 0000000..09e6514
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/operation.cc
@@ -0,0 +1,150 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/operation.h"
+
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/dbi/hiveserver2/ImpalaService_types.h"
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/macros.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+using std::unique_ptr;
+
+namespace arrow {
+namespace hiveserver2 {
+
+// Max rows to fetch, if not specified.
+constexpr int kDefaultMaxRows = 1024;
+
+Operation::Operation(const std::shared_ptr<ThriftRPC>& rpc)
+    : impl_(new OperationImpl()), rpc_(rpc), open_(false) {}
+
+Operation::~Operation() { DCHECK(!open_); }
+
+Status Operation::GetState(Operation::State* out) const {
+  hs2::TGetOperationStatusReq req;
+  req.__set_operationHandle(impl_->handle);
+  hs2::TGetOperationStatusResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->GetOperationStatus(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+  *out = TOperationStateToOperationState(resp.operationState);
+  return TStatusToStatus(resp.status);
+}
+
+Status Operation::GetLog(std::string* out) const {
+  hs2::TGetLogReq req;
+  req.__set_operationHandle(impl_->handle);
+  hs2::TGetLogResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->GetLog(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+  *out = resp.log;
+  return TStatusToStatus(resp.status);
+}
+
+Status Operation::GetProfile(std::string* out) const {
+  impala::TGetRuntimeProfileReq req;
+  req.__set_operationHandle(impl_->handle);
+  req.__set_sessionHandle(impl_->session_handle);
+  impala::TGetRuntimeProfileResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->GetRuntimeProfile(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+  *out = resp.profile;
+  return TStatusToStatus(resp.status);
+}
+
+Status Operation::GetResultSetMetadata(std::vector<ColumnDesc>* column_descs) const {
+  hs2::TGetResultSetMetadataReq req;
+  req.__set_operationHandle(impl_->handle);
+  hs2::TGetResultSetMetadataResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->GetResultSetMetadata(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+
+  column_descs->clear();
+  column_descs->reserve(resp.schema.columns.size());
+  for (const hs2::TColumnDesc& tcolumn_desc : resp.schema.columns) {
+    column_descs->emplace_back(tcolumn_desc.columnName,
+                               TTypeDescToColumnType(tcolumn_desc.typeDesc),
+                               tcolumn_desc.position, tcolumn_desc.comment);
+  }
+
+  return TStatusToStatus(resp.status);
+}
+
+Status Operation::Fetch(unique_ptr<ColumnarRowSet>* results, bool* has_more_rows) const {
+  return Fetch(kDefaultMaxRows, FetchOrientation::NEXT, results, has_more_rows);
+}
+
+Status Operation::Fetch(int max_rows, FetchOrientation orientation,
+                        unique_ptr<ColumnarRowSet>* results, bool* has_more_rows) const {
+  hs2::TFetchResultsReq req;
+  req.__set_operationHandle(impl_->handle);
+  req.__set_orientation(FetchOrientationToTFetchOrientation(orientation));
+  req.__set_maxRows(max_rows);
+  std::unique_ptr<ColumnarRowSet::ColumnarRowSetImpl> row_set_impl(
+      new ColumnarRowSet::ColumnarRowSetImpl());
+  TRY_RPC_OR_RETURN(rpc_->client->FetchResults(row_set_impl->resp, req));
+  THRIFT_RETURN_NOT_OK(row_set_impl->resp.status);
+
+  if (has_more_rows != NULL) {
+    *has_more_rows = row_set_impl->resp.hasMoreRows;
+  }
+  Status status = TStatusToStatus(row_set_impl->resp.status);
+  DCHECK(status.ok());
+  results->reset(new ColumnarRowSet(row_set_impl.release()));
+  return status;
+}
+
+Status Operation::Cancel() const {
+  hs2::TCancelOperationReq req;
+  req.__set_operationHandle(impl_->handle);
+  hs2::TCancelOperationResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->CancelOperation(resp, req));
+  return TStatusToStatus(resp.status);
+}
+
+Status Operation::Close() {
+  if (!open_) return Status::OK();
+
+  hs2::TCloseOperationReq req;
+  req.__set_operationHandle(impl_->handle);
+  hs2::TCloseOperationResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->CloseOperation(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+
+  open_ = false;
+  return TStatusToStatus(resp.status);
+}
+
+bool Operation::HasResultSet() const {
+  State op_state;
+  Status s = GetState(&op_state);
+  if (!s.ok()) return false;
+  return op_state == State::FINISHED;
+}
+
+bool Operation::IsColumnar() const {
+  // We currently only support the columnar hs2 protocols.
+  return true;
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/operation.h b/cpp/src/arrow/dbi/hiveserver2/operation.h
new file mode 100644
index 0000000..f275592
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/operation.h
@@ -0,0 +1,127 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/dbi/hiveserver2/columnar-row-set.h"
+#include "arrow/dbi/hiveserver2/types.h"
+
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Status;
+
+namespace hiveserver2 {
+
+struct ThriftRPC;
+
+// Maps directly to TFetchOrientation in the HiveServer2 interface.
+enum class FetchOrientation {
+  NEXT,      // supported
+  PRIOR,     // not supported
+  RELATIVE,  // not supported
+  ABSOLUTE,  // not supported
+  FIRST,     // supported if query result caching is enabled in Impala
+  LAST       // not supported
+};
+
+// Represents a single HiveServer2 operation. Used to monitor the status of an operation
+// and to retrieve its results. The only Operation function that will block is Fetch,
+// which blocks if there aren't any results ready yet.
+//
+// Operations are created using Session functions, eg. ExecuteStatement. They must
+// have Close called on them before they can be deleted.
+//
+// This class is not thread-safe.
+class ARROW_EXPORT Operation {
+ public:
+  // Maps directly to TOperationState in the HiveServer2 interface.
+  enum class State {
+    INITIALIZED,
+    RUNNING,
+    FINISHED,
+    CANCELED,
+    CLOSED,
+    ERROR,
+    UNKNOWN,
+    PENDING,
+  };
+
+  ~Operation();
+
+  // Fetches the current state of this operation. If successful, sets the operation state
+  // in 'out' and returns an OK status, otherwise an error status is returned. May be
+  // called after successfully creating the operation and before calling Close.
+  Status GetState(Operation::State* out) const;
+
+  // May be called after successfully creating the operation and before calling Close.
+  Status GetLog(std::string* out) const;
+
+  // May be called after successfully creating the operation and before calling Close.
+  Status GetProfile(std::string* out) const;
+
+  // Fetches metadata for the columns in the output of this operation, such as the
+  // names and types of the columns, and returns it as a list of column descriptions.
+  // May be called after successfully creating the operation and before calling Close.
+  Status GetResultSetMetadata(std::vector<ColumnDesc>* column_descs) const;
+
+  // Fetches a batch of results, stores them in 'results', and sets has_more_rows.
+  // Fetch will block if there aren't any results that are ready.
+  Status Fetch(std::unique_ptr<ColumnarRowSet>* results, bool* has_more_rows) const;
+  Status Fetch(int max_rows, FetchOrientation orientation,
+               std::unique_ptr<ColumnarRowSet>* results, bool* has_more_rows) const;
+
+  // May be called after successfully creating the operation and before calling Close.
+  Status Cancel() const;
+
+  // Closes the operation. Must be called before the operation is deleted. May be safely
+  // called on an invalid or already closed operation - will only return an error if the
+  // operation is open but the close rpc fails.
+  Status Close();
+
+  // May be called after successfully creating the operation and before calling Close.
+  bool HasResultSet() const;
+
+  // Returns true iff this operation's results will be returned in a columnar format.
+  // May be called at any time.
+  bool IsColumnar() const;
+
+ protected:
+  // Hides Thrift objects from the header.
+  struct OperationImpl;
+
+  explicit Operation(const std::shared_ptr<ThriftRPC>& rpc);
+
+  std::unique_ptr<OperationImpl> impl_;
+  std::shared_ptr<ThriftRPC> rpc_;
+
+  // True iff this operation has been successfully created and has not been closed yet,
+  // corresponding to when the operation has a valid operation handle.
+  bool open_;
+
+ private:
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Operation);
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/public-api-test.cc b/cpp/src/arrow/dbi/hiveserver2/public-api-test.cc
new file mode 100644
index 0000000..833ad02
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/public-api-test.cc
@@ -0,0 +1,26 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include "arrow/adapters/hiveserver2/api.h"
+
+TEST(TestPublicAPI, DoesNotIncludeThrift) {
+#ifdef _THRIFT_THRIFT_H_
+  FAIL() << "Thrift headers should not be in the public API";
+#endif
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/sample-usage.cc b/cpp/src/arrow/dbi/hiveserver2/sample-usage.cc
new file mode 100644
index 0000000..f16a81b
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/sample-usage.cc
@@ -0,0 +1,139 @@
+// 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.
+
+#include <cassert>
+#include <iostream>
+#include <memory>
+#include <string>
+
+#include "arrow/dbi/hiveserver2/api.h"
+
+namespace hs2 = arrow::hiveserver2;
+
+using arrow::Status;
+using std::string;
+using std::unique_ptr;
+
+using hs2::Operation;
+using hs2::Service;
+using hs2::Session;
+
+#define ABORT_NOT_OK(s)                  \
+  do {                                   \
+    ::arrow::Status _s = (s);            \
+    if (ARROW_PREDICT_FALSE(!_s.ok())) { \
+      std::cerr << s.ToString() << "\n"; \
+      std::abort();                      \
+    }                                    \
+  } while (false);
+
+int main(int argc, char** argv) {
+  // Connect to the server.
+  string host = "localhost";
+  int port = 21050;
+  int conn_timeout = 0;
+  hs2::ProtocolVersion protocol = hs2::ProtocolVersion::PROTOCOL_V7;
+  unique_ptr<Service> service;
+  Status status = Service::Connect(host, port, conn_timeout, protocol, &service);
+  if (!status.ok()) {
+    std::cout << "Failed to connect to service: " << status.ToString();
+    ABORT_NOT_OK(service->Close());
+    return 1;
+  }
+
+  // Open a session.
+  string user = "user";
+  hs2::HS2ClientConfig config;
+  unique_ptr<Session> session;
+  status = service->OpenSession(user, config, &session);
+  if (!status.ok()) {
+    std::cout << "Failed to open session: " << status.ToString();
+    ABORT_NOT_OK(session->Close());
+    ABORT_NOT_OK(service->Close());
+    return 1;
+  }
+
+  // Execute a statement.
+  string statement = "SELECT int_col, string_col FROM test order by int_col";
+  unique_ptr<hs2::Operation> execute_op;
+  status = session->ExecuteStatement(statement, &execute_op);
+  if (!status.ok()) {
+    std::cout << "Failed to execute select: " << status.ToString();
+    ABORT_NOT_OK(execute_op->Close());
+    ABORT_NOT_OK(session->Close());
+    ABORT_NOT_OK(service->Close());
+    return 1;
+  }
+
+  unique_ptr<hs2::ColumnarRowSet> execute_results;
+  bool has_more_rows = true;
+  int total_retrieved = 0;
+  std::cout << "Contents of test:\n";
+  while (has_more_rows) {
+    status = execute_op->Fetch(&execute_results, &has_more_rows);
+    if (!status.ok()) {
+      std::cout << "Failed to fetch results: " << status.ToString();
+      ABORT_NOT_OK(execute_op->Close());
+      ABORT_NOT_OK(session->Close());
+      ABORT_NOT_OK(service->Close());
+      return 1;
+    }
+
+    unique_ptr<hs2::Int32Column> int_col = execute_results->GetInt32Col(0);
+    unique_ptr<hs2::StringColumn> string_col = execute_results->GetStringCol(1);
+    assert(int_col->length() == string_col->length());
+    total_retrieved += int_col->length();
+    for (int64_t i = 0; i < int_col->length(); ++i) {
+      if (int_col->IsNull(i)) {
+        std::cout << "NULL";
+      } else {
+        std::cout << int_col->GetData(i);
+      }
+      std::cout << ":";
+
+      if (string_col->IsNull(i)) {
+        std::cout << "NULL";
+      } else {
+        std::cout << "'" << string_col->GetData(i) << "'";
+      }
+      std::cout << "\n";
+    }
+  }
+  std::cout << "retrieved " << total_retrieved << " rows\n";
+  std::cout << "\n";
+  ABORT_NOT_OK(execute_op->Close());
+
+  unique_ptr<Operation> show_tables_op;
+  status = session->ExecuteStatement("show tables", &show_tables_op);
+  if (!status.ok()) {
+    std::cout << "Failed to execute GetTables: " << status.ToString();
+    ABORT_NOT_OK(show_tables_op->Close());
+    ABORT_NOT_OK(session->Close());
+    ABORT_NOT_OK(service->Close());
+    return 1;
+  }
+
+  std::cout << "Show tables:\n";
+  hs2::Util::PrintResults(show_tables_op.get(), std::cout);
+  ABORT_NOT_OK(show_tables_op->Close());
+
+  // Shut down.
+  ABORT_NOT_OK(session->Close());
+  ABORT_NOT_OK(service->Close());
+
+  return 0;
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/service.cc b/cpp/src/arrow/dbi/hiveserver2/service.cc
new file mode 100644
index 0000000..e2d3f2a
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/service.cc
@@ -0,0 +1,113 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/service.h"
+
+#include <thrift/protocol/TBinaryProtocol.h>
+#include <thrift/transport/TSocket.h>
+#include <thrift/transport/TTransportUtils.h>
+#include <sstream>
+
+#include "arrow/dbi/hiveserver2/session.h"
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/dbi/hiveserver2/ImpalaHiveServer2Service.h"
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+
+using apache::thrift::TException;
+using apache::thrift::protocol::TBinaryProtocol;
+using apache::thrift::protocol::TProtocol;
+using apache::thrift::transport::TBufferedTransport;
+using apache::thrift::transport::TSocket;
+using apache::thrift::transport::TTransport;
+using std::string;
+using std::unique_ptr;
+
+namespace arrow {
+namespace hiveserver2 {
+
+struct Service::ServiceImpl {
+  hs2::TProtocolVersion::type protocol_version;
+  std::shared_ptr<TSocket> socket;
+  std::shared_ptr<TTransport> transport;
+  std::shared_ptr<TProtocol> protocol;
+};
+
+Status Service::Connect(const string& host, int port, int conn_timeout,
+                        ProtocolVersion protocol_version, unique_ptr<Service>* service) {
+  service->reset(new Service(host, port, conn_timeout, protocol_version));
+  return (*service)->Open();
+}
+
+Service::~Service() { DCHECK(!IsConnected()); }
+
+Status Service::Close() {
+  if (!IsConnected()) return Status::OK();
+  TRY_RPC_OR_RETURN(impl_->transport->close());
+  return Status::OK();
+}
+
+bool Service::IsConnected() const {
+  return impl_->transport && impl_->transport->isOpen();
+}
+
+void Service::SetRecvTimeout(int timeout) { impl_->socket->setRecvTimeout(timeout); }
+
+void Service::SetSendTimeout(int timeout) { impl_->socket->setSendTimeout(timeout); }
+
+Status Service::OpenSession(const string& user, const HS2ClientConfig& config,
+                            unique_ptr<Session>* session) const {
+  session->reset(new Session(rpc_));
+  return (*session)->Open(config, user);
+}
+
+Service::Service(const string& host, int port, int conn_timeout,
+                 ProtocolVersion protocol_version)
+    : host_(host),
+      port_(port),
+      conn_timeout_(conn_timeout),
+      impl_(new ServiceImpl()),
+      rpc_(new ThriftRPC()) {
+  impl_->protocol_version = ProtocolVersionToTProtocolVersion(protocol_version);
+}
+
+Status Service::Open() {
+  if (impl_->protocol_version < hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6) {
+    std::stringstream ss;
+    ss << "Unsupported protocol: " << impl_->protocol_version;
+    return Status::NotImplemented(ss.str());
+  }
+
+  impl_->socket.reset(new TSocket(host_, port_));
+  impl_->socket->setConnTimeout(conn_timeout_);
+  impl_->transport.reset(new TBufferedTransport(impl_->socket));
+  impl_->protocol.reset(new TBinaryProtocol(impl_->transport));
+
+  rpc_->client.reset(new impala::ImpalaHiveServer2ServiceClient(impl_->protocol));
+
+  TRY_RPC_OR_RETURN(impl_->transport->open());
+
+  return Status::OK();
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/service.h b/cpp/src/arrow/dbi/hiveserver2/service.h
new file mode 100644
index 0000000..bfa7a97
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/service.h
@@ -0,0 +1,140 @@
+// 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.
+
+#pragma once
+
+#include <map>
+#include <memory>
+#include <string>
+
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Status;
+
+namespace hiveserver2 {
+
+class Session;
+struct ThriftRPC;
+
+// Stores per-session or per-operation configuration parameters.
+class HS2ClientConfig {
+ public:
+  void SetOption(const std::string& key, const std::string& value) {
+    config_[key] = value;
+  }
+
+  bool GetOption(const std::string& key, std::string* value_out) {
+    if (config_.find(key) != config_.end() && value_out) {
+      *value_out = config_[key];
+      return true;
+    }
+    return false;
+  }
+
+  const std::map<std::string, std::string>& GetConfig() const { return config_; }
+
+ private:
+  std::map<std::string, std::string> config_;
+};
+
+// Maps directly to TProtocolVersion in the HiveServer2 interface.
+enum class ProtocolVersion {
+  PROTOCOL_V1,  // not supported
+  PROTOCOL_V2,  // not supported
+  PROTOCOL_V3,  // not supported
+  PROTOCOL_V4,  // not supported
+  PROTOCOL_V5,  // not supported
+  PROTOCOL_V6,  // supported
+  PROTOCOL_V7,  // supported
+};
+
+// Manages a connection to a HiveServer2 server. Primarily used to create
+// new sessions via OpenSession.
+//
+// Service objects are created using Service::Connect(). They must
+// have Close called on them before they can be deleted.
+//
+// This class is not thread-safe.
+//
+// Example:
+// unique_ptr<Service> service;
+// if (Service::Connect(host, port, protocol_version, &service).ok()) {
+//   // do some work
+//   service->Close();
+// }
+class ARROW_EXPORT Service {
+ public:
+  // Creates a new connection to a HS2 service at the given host and port. If
+  // conn_timeout > 0, connection attempts will timeout after conn_timeout ms, otherwise
+  // no timeout is used. protocol_version is the HiveServer2 protocol to use, and
+  // determines whether the results returned by operations from this service are row or
+  // column oriented. Only column oriented protocols are currently supported.
+  //
+  // The client calling Connect has ownership of the new Service that is created.
+  // Executing RPCs with an Session or Operation corresponding to a particular
+  // Service after that Service has been closed or deleted in undefined.
+  static Status Connect(const std::string& host, int port, int conn_timeout,
+                        ProtocolVersion protocol_version,
+                        std::unique_ptr<Service>* service);
+
+  ~Service();
+
+  // Closes the connection. Must be called before the service is deleted. May be
+  // safely called on an invalid or already closed service - will only return an
+  // error if the service is open but the close rpc fails.
+  Status Close();
+
+  // Returns true iff this service has an active connection to the HiveServer2 server.
+  bool IsConnected() const;
+
+  // Set the send and receive timeout for Thrift RPCs in ms. 0 indicates no timeout,
+  // negative values are ignored.
+  void SetRecvTimeout(int timeout);
+  void SetSendTimeout(int timeout);
+
+  // Opens a new HS2 session using this service.
+  // The client calling OpenSession has ownership of the Session that is created.
+  // Operations on the Session are undefined once it is closed.
+  Status OpenSession(const std::string& user, const HS2ClientConfig& config,
+                     std::unique_ptr<Session>* session) const;
+
+ private:
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Service);
+
+  // Hides Thrift objects from the header.
+  struct ServiceImpl;
+
+  Service(const std::string& host, int port, int conn_timeout,
+          ProtocolVersion protocol_version);
+
+  // Opens the connection to the server. Called by Connect before new service is returned
+  // to the user. Must be called before OpenSession.
+  Status Open();
+
+  std::string host_;
+  int port_;
+  int conn_timeout_;
+
+  std::unique_ptr<ServiceImpl> impl_;
+  std::shared_ptr<ThriftRPC> rpc_;
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/session.cc b/cpp/src/arrow/dbi/hiveserver2/session.cc
new file mode 100644
index 0000000..0d37848
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/session.cc
@@ -0,0 +1,105 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/session.h"
+
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+using apache::thrift::TException;
+using std::string;
+using std::unique_ptr;
+
+namespace arrow {
+namespace hiveserver2 {
+
+struct Session::SessionImpl {
+  hs2::TSessionHandle handle;
+};
+
+Session::Session(const std::shared_ptr<ThriftRPC>& rpc)
+    : impl_(new SessionImpl()), rpc_(rpc), open_(false) {}
+
+Session::~Session() { DCHECK(!open_); }
+
+Status Session::Close() {
+  if (!open_) return Status::OK();
+
+  hs2::TCloseSessionReq req;
+  req.__set_sessionHandle(impl_->handle);
+  hs2::TCloseSessionResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->CloseSession(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+
+  open_ = false;
+  return TStatusToStatus(resp.status);
+}
+
+Status Session::Open(const HS2ClientConfig& config, const string& user) {
+  hs2::TOpenSessionReq req;
+  req.__set_configuration(config.GetConfig());
+  req.__set_username(user);
+  hs2::TOpenSessionResp resp;
+  TRY_RPC_OR_RETURN(rpc_->client->OpenSession(resp, req));
+  THRIFT_RETURN_NOT_OK(resp.status);
+
+  impl_->handle = resp.sessionHandle;
+  open_ = true;
+  return TStatusToStatus(resp.status);
+}
+
+class ExecuteStatementOperation : public Operation {
+ public:
+  explicit ExecuteStatementOperation(const std::shared_ptr<ThriftRPC>& rpc)
+      : Operation(rpc) {}
+
+  Status Open(hs2::TSessionHandle session_handle, const string& statement,
+              const HS2ClientConfig& config) {
+    hs2::TExecuteStatementReq req;
+    req.__set_sessionHandle(session_handle);
+    req.__set_statement(statement);
+    req.__set_confOverlay(config.GetConfig());
+    hs2::TExecuteStatementResp resp;
+    TRY_RPC_OR_RETURN(rpc_->client->ExecuteStatement(resp, req));
+    THRIFT_RETURN_NOT_OK(resp.status);
+
+    impl_->handle = resp.operationHandle;
+    impl_->session_handle = session_handle;
+    open_ = true;
+    return TStatusToStatus(resp.status);
+  }
+};
+
+Status Session::ExecuteStatement(const string& statement,
+                                 unique_ptr<Operation>* operation) const {
+  return ExecuteStatement(statement, HS2ClientConfig(), operation);
+}
+
+Status Session::ExecuteStatement(const string& statement,
+                                 const HS2ClientConfig& conf_overlay,
+                                 unique_ptr<Operation>* operation) const {
+  ExecuteStatementOperation* op = new ExecuteStatementOperation(rpc_);
+  operation->reset(op);
+  return op->Open(impl_->handle, statement, conf_overlay);
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/session.h b/cpp/src/arrow/dbi/hiveserver2/session.h
new file mode 100644
index 0000000..4e223de
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/session.h
@@ -0,0 +1,84 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "arrow/dbi/hiveserver2/operation.h"
+#include "arrow/dbi/hiveserver2/service.h"
+
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Status;
+
+namespace hiveserver2 {
+
+struct ThriftRPC;
+
+// Manages a single HiveServer2 session - stores the session handle returned by
+// the OpenSession RPC and uses it to create and return operations.
+//
+// Sessions are created with Service::OpenSession(). They must have Close
+// called on them before they can be deleted.
+//
+// Executing RPCs with an Operation corresponding to a particular Session after
+// that Session has been closed or deleted is undefined.
+//
+// This class is not thread-safe.
+class ARROW_EXPORT Session {
+ public:
+  ~Session();
+
+  // Closes the session. Must be called before the session is deleted. May be safely
+  // called on an invalid or already closed session - will only return an error if the
+  // session is open but the close rpc fails.
+  Status Close();
+
+  Status ExecuteStatement(const std::string& statement,
+                          std::unique_ptr<Operation>* operation) const;
+  Status ExecuteStatement(const std::string& statement,
+                          const HS2ClientConfig& conf_overlay,
+                          std::unique_ptr<Operation>* operation) const;
+
+ private:
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Session);
+
+  // Hides Thrift objects from the header.
+  struct SessionImpl;
+
+  // For access to the c'tor.
+  friend class Service;
+
+  explicit Session(const std::shared_ptr<ThriftRPC>& rpc);
+
+  // Performs the RPC that initiates the session and stores the returned handle.
+  // Must be called before operations can be executed.
+  Status Open(const HS2ClientConfig& config, const std::string& user);
+
+  std::unique_ptr<SessionImpl> impl_;
+  std::shared_ptr<ThriftRPC> rpc_;
+
+  // True if Open has been called and Close has not.
+  bool open_;
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc
new file mode 100644
index 0000000..d154e14
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc
@@ -0,0 +1,303 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include <map>
+#include <sstream>
+
+#include "arrow/dbi/hiveserver2/TCLIService_constants.h"
+#include "arrow/dbi/hiveserver2/service.h"
+
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+
+namespace arrow {
+namespace hiveserver2 {
+
+namespace {
+
+// Convert an "enum class" value to an integer equivalent, for outputting.
+template <typename ENUM>
+typename std::underlying_type<ENUM>::type EnumToInt(const ENUM& value) {
+  return static_cast<typename std::underlying_type<ENUM>::type>(value);
+}
+
+}  // namespace
+
+const std::string OperationStateToString(const Operation::State& state) {
+  switch (state) {
+    case Operation::State::INITIALIZED:
+      return "INITIALIZED";
+    case Operation::State::RUNNING:
+      return "RUNNING";
+    case Operation::State::FINISHED:
+      return "FINISHED";
+    case Operation::State::CANCELED:
+      return "CANCELED";
+    case Operation::State::CLOSED:
+      return "CLOSED";
+    case Operation::State::ERROR:
+      return "ERROR";
+    case Operation::State::UNKNOWN:
+      return "UNKNOWN";
+    case Operation::State::PENDING:
+      return "PENDING";
+    default:
+      std::stringstream ss;
+      ss << "Unknown Operation::State " << EnumToInt(state);
+      return ss.str();
+  }
+}
+
+const std::string TypeIdToString(const ColumnType::TypeId& type_id) {
+  switch (type_id) {
+    case ColumnType::TypeId::BOOLEAN:
+      return "BOOLEAN";
+    case ColumnType::TypeId::TINYINT:
+      return "TINYINT";
+    case ColumnType::TypeId::SMALLINT:
+      return "SMALLINT";
+    case ColumnType::TypeId::INT:
+      return "INT";
+    case ColumnType::TypeId::BIGINT:
+      return "BIGINT";
+    case ColumnType::TypeId::FLOAT:
+      return "FLOAT";
+    case ColumnType::TypeId::DOUBLE:
+      return "DOUBLE";
+    case ColumnType::TypeId::STRING:
+      return "STRING";
+    case ColumnType::TypeId::TIMESTAMP:
+      return "TIMESTAMP";
+    case ColumnType::TypeId::BINARY:
+      return "BINARY";
+    case ColumnType::TypeId::ARRAY:
+      return "ARRAY";
+    case ColumnType::TypeId::MAP:
+      return "MAP";
+    case ColumnType::TypeId::STRUCT:
+      return "STRUCT";
+    case ColumnType::TypeId::UNION:
+      return "UNION";
+    case ColumnType::TypeId::USER_DEFINED:
+      return "USER_DEFINED";
+    case ColumnType::TypeId::DECIMAL:
+      return "DECIMAL";
+    case ColumnType::TypeId::NULL_TYPE:
+      return "NULL_TYPE";
+    case ColumnType::TypeId::DATE:
+      return "DATE";
+    case ColumnType::TypeId::VARCHAR:
+      return "VARCHAR";
+    case ColumnType::TypeId::CHAR:
+      return "CHAR";
+    case ColumnType::TypeId::INVALID:
+      return "INVALID";
+    default: {
+      std::stringstream ss;
+      ss << "Unknown ColumnType::TypeId " << EnumToInt(type_id);
+      return ss.str();
+    }
+  }
+}
+
+hs2::TFetchOrientation::type FetchOrientationToTFetchOrientation(
+    FetchOrientation orientation) {
+  switch (orientation) {
+    case FetchOrientation::NEXT:
+      return hs2::TFetchOrientation::FETCH_NEXT;
+    case FetchOrientation::PRIOR:
+      return hs2::TFetchOrientation::FETCH_PRIOR;
+    case FetchOrientation::RELATIVE:
+      return hs2::TFetchOrientation::FETCH_RELATIVE;
+    case FetchOrientation::ABSOLUTE:
+      return hs2::TFetchOrientation::FETCH_ABSOLUTE;
+    case FetchOrientation::FIRST:
+      return hs2::TFetchOrientation::FETCH_FIRST;
+    case FetchOrientation::LAST:
+      return hs2::TFetchOrientation::FETCH_LAST;
+    default:
+      DCHECK(false) << "Unknown FetchOrientation " << EnumToInt(orientation);
+      return hs2::TFetchOrientation::FETCH_NEXT;
+  }
+}
+
+hs2::TProtocolVersion::type ProtocolVersionToTProtocolVersion(ProtocolVersion protocol) {
+  switch (protocol) {
+    case ProtocolVersion::PROTOCOL_V1:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V1;
+    case ProtocolVersion::PROTOCOL_V2:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V2;
+    case ProtocolVersion::PROTOCOL_V3:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V3;
+    case ProtocolVersion::PROTOCOL_V4:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V4;
+    case ProtocolVersion::PROTOCOL_V5:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V5;
+    case ProtocolVersion::PROTOCOL_V6:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6;
+    case ProtocolVersion::PROTOCOL_V7:
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V7;
+    default:
+      DCHECK(false) << "Unknown ProtocolVersion " << EnumToInt(protocol);
+      return hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V7;
+  }
+}
+
+Operation::State TOperationStateToOperationState(
+    const hs2::TOperationState::type& tstate) {
+  switch (tstate) {
+    case hs2::TOperationState::INITIALIZED_STATE:
+      return Operation::State::INITIALIZED;
+    case hs2::TOperationState::RUNNING_STATE:
+      return Operation::State::RUNNING;
+    case hs2::TOperationState::FINISHED_STATE:
+      return Operation::State::FINISHED;
+    case hs2::TOperationState::CANCELED_STATE:
+      return Operation::State::CANCELED;
+    case hs2::TOperationState::CLOSED_STATE:
+      return Operation::State::CLOSED;
+    case hs2::TOperationState::ERROR_STATE:
+      return Operation::State::ERROR;
+    case hs2::TOperationState::UKNOWN_STATE:
+      return Operation::State::UNKNOWN;
+    case hs2::TOperationState::PENDING_STATE:
+      return Operation::State::PENDING;
+    default:
+      ARROW_LOG(WARNING) << "Unknown TOperationState " << tstate;
+      return Operation::State::UNKNOWN;
+  }
+}
+
+Status TStatusToStatus(const hs2::TStatus& tstatus) {
+  switch (tstatus.statusCode) {
+    case hs2::TStatusCode::SUCCESS_STATUS:
+      return Status::OK();
+    case hs2::TStatusCode::SUCCESS_WITH_INFO_STATUS: {
+      std::stringstream ss;
+      for (size_t i = 0; i < tstatus.infoMessages.size(); i++) {
+        if (i != 0) ss << ",";
+        ss << tstatus.infoMessages[i];
+      }
+      return Status::OK(ss.str());
+    }
+    case hs2::TStatusCode::STILL_EXECUTING_STATUS:
+      return Status::StillExecuting();
+    case hs2::TStatusCode::ERROR_STATUS:
+      return Status::IOError(tstatus.errorMessage);
+    case hs2::TStatusCode::INVALID_HANDLE_STATUS:
+      return Status::Invalid("Invalid handle");
+    default: {
+      std::stringstream ss;
+      ss << "Unknown TStatusCode " << tstatus.statusCode;
+      return Status::UnknownError(ss.str());
+    }
+  }
+}
+
+std::unique_ptr<ColumnType> TTypeDescToColumnType(const hs2::TTypeDesc& ttype_desc) {
+  if (ttype_desc.types.size() != 1 || !ttype_desc.types[0].__isset.primitiveEntry) {
+    ARROW_LOG(WARNING) << "TTypeDescToColumnType only supports primitive types.";
+    return std::unique_ptr<ColumnType>(new PrimitiveType(ColumnType::TypeId::INVALID));
+  }
+
+  ColumnType::TypeId type_id = TTypeIdToTypeId(ttype_desc.types[0].primitiveEntry.type);
+  if (type_id == ColumnType::TypeId::CHAR || type_id == ColumnType::TypeId::VARCHAR) {
+    const std::map<std::string, hs2::TTypeQualifierValue>& qualifiers =
+        ttype_desc.types[0].primitiveEntry.typeQualifiers.qualifiers;
+    DCHECK_EQ(qualifiers.count(hs2::g_TCLIService_constants.CHARACTER_MAXIMUM_LENGTH), 1);
+
+    try {
+      return std::unique_ptr<ColumnType>(new CharacterType(
+          type_id,
+          qualifiers.at(hs2::g_TCLIService_constants.CHARACTER_MAXIMUM_LENGTH).i32Value));
+    } catch (std::out_of_range e) {
+      ARROW_LOG(ERROR) << "Character type qualifiers invalid: " << e.what();
+      return std::unique_ptr<ColumnType>(new PrimitiveType(ColumnType::TypeId::INVALID));
+    }
+  } else if (type_id == ColumnType::TypeId::DECIMAL) {
+    const std::map<std::string, hs2::TTypeQualifierValue>& qualifiers =
+        ttype_desc.types[0].primitiveEntry.typeQualifiers.qualifiers;
+    DCHECK_EQ(qualifiers.count(hs2::g_TCLIService_constants.PRECISION), 1);
+    DCHECK_EQ(qualifiers.count(hs2::g_TCLIService_constants.SCALE), 1);
+
+    try {
+      return std::unique_ptr<ColumnType>(new DecimalType(
+          type_id, qualifiers.at(hs2::g_TCLIService_constants.PRECISION).i32Value,
+          qualifiers.at(hs2::g_TCLIService_constants.SCALE).i32Value));
+    } catch (std::out_of_range e) {
+      ARROW_LOG(ERROR) << "Decimal type qualifiers invalid: " << e.what();
+      return std::unique_ptr<ColumnType>(new PrimitiveType(ColumnType::TypeId::INVALID));
+    }
+  } else {
+    return std::unique_ptr<ColumnType>(new PrimitiveType(type_id));
+  }
+}
+
+ColumnType::TypeId TTypeIdToTypeId(const hs2::TTypeId::type& type_id) {
+  switch (type_id) {
+    case hs2::TTypeId::BOOLEAN_TYPE:
+      return ColumnType::TypeId::BOOLEAN;
+    case hs2::TTypeId::TINYINT_TYPE:
+      return ColumnType::TypeId::TINYINT;
+    case hs2::TTypeId::SMALLINT_TYPE:
+      return ColumnType::TypeId::SMALLINT;
+    case hs2::TTypeId::INT_TYPE:
+      return ColumnType::TypeId::INT;
+    case hs2::TTypeId::BIGINT_TYPE:
+      return ColumnType::TypeId::BIGINT;
+    case hs2::TTypeId::FLOAT_TYPE:
+      return ColumnType::TypeId::FLOAT;
+    case hs2::TTypeId::DOUBLE_TYPE:
+      return ColumnType::TypeId::DOUBLE;
+    case hs2::TTypeId::STRING_TYPE:
+      return ColumnType::TypeId::STRING;
+    case hs2::TTypeId::TIMESTAMP_TYPE:
+      return ColumnType::TypeId::TIMESTAMP;
+    case hs2::TTypeId::BINARY_TYPE:
+      return ColumnType::TypeId::BINARY;
+    case hs2::TTypeId::ARRAY_TYPE:
+      return ColumnType::TypeId::ARRAY;
+    case hs2::TTypeId::MAP_TYPE:
+      return ColumnType::TypeId::MAP;
+    case hs2::TTypeId::STRUCT_TYPE:
+      return ColumnType::TypeId::STRUCT;
+    case hs2::TTypeId::UNION_TYPE:
+      return ColumnType::TypeId::UNION;
+    case hs2::TTypeId::USER_DEFINED_TYPE:
+      return ColumnType::TypeId::USER_DEFINED;
+    case hs2::TTypeId::DECIMAL_TYPE:
+      return ColumnType::TypeId::DECIMAL;
+    case hs2::TTypeId::NULL_TYPE:
+      return ColumnType::TypeId::NULL_TYPE;
+    case hs2::TTypeId::DATE_TYPE:
+      return ColumnType::TypeId::DATE;
+    case hs2::TTypeId::VARCHAR_TYPE:
+      return ColumnType::TypeId::VARCHAR;
+    case hs2::TTypeId::CHAR_TYPE:
+      return ColumnType::TypeId::CHAR;
+    default:
+      ARROW_LOG(WARNING) << "Unknown TTypeId " << type_id;
+      return ColumnType::TypeId::INVALID;
+  }
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift-internal.h b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.h
new file mode 100644
index 0000000..aad535f
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.h
@@ -0,0 +1,91 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "arrow/dbi/hiveserver2/columnar-row-set.h"
+#include "arrow/dbi/hiveserver2/operation.h"
+#include "arrow/dbi/hiveserver2/service.h"
+#include "arrow/dbi/hiveserver2/types.h"
+
+#include "arrow/dbi/hiveserver2/ImpalaHiveServer2Service.h"
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+
+namespace arrow {
+namespace hiveserver2 {
+
+// PIMPL structs.
+struct ColumnarRowSet::ColumnarRowSetImpl {
+  apache::hive::service::cli::thrift::TFetchResultsResp resp;
+};
+
+struct Operation::OperationImpl {
+  apache::hive::service::cli::thrift::TOperationHandle handle;
+  apache::hive::service::cli::thrift::TSessionHandle session_handle;
+};
+
+struct ThriftRPC {
+  std::unique_ptr<impala::ImpalaHiveServer2ServiceClient> client;
+};
+
+const std::string OperationStateToString(const Operation::State& state);
+
+const std::string TypeIdToString(const ColumnType::TypeId& type_id);
+
+// Functions for converting Thrift object to hs2client objects and vice-versa.
+apache::hive::service::cli::thrift::TFetchOrientation::type
+FetchOrientationToTFetchOrientation(FetchOrientation orientation);
+
+apache::hive::service::cli::thrift::TProtocolVersion::type
+ProtocolVersionToTProtocolVersion(ProtocolVersion protocol);
+
+Operation::State TOperationStateToOperationState(
+    const apache::hive::service::cli::thrift::TOperationState::type& tstate);
+
+Status TStatusToStatus(const apache::hive::service::cli::thrift::TStatus& tstatus);
+
+// Converts a TTypeDesc to a ColumnType. Currently only primitive types are supported.
+// The converted type is returned as a pointer to allow for polymorphism with ColumnType
+// and its subclasses.
+std::unique_ptr<ColumnType> TTypeDescToColumnType(
+    const apache::hive::service::cli::thrift::TTypeDesc& ttype_desc);
+
+ColumnType::TypeId TTypeIdToTypeId(
+    const apache::hive::service::cli::thrift::TTypeId::type& type_id);
+
+}  // namespace hiveserver2
+}  // namespace arrow
+
+#define TRY_RPC_OR_RETURN(rpc)                  \
+  do {                                          \
+    try {                                       \
+      (rpc);                                    \
+    } catch (apache::thrift::TException & tx) { \
+      return Status::IOError(tx.what());        \
+    }                                           \
+  } while (0)
+
+#define THRIFT_RETURN_NOT_OK(tstatus)                                       \
+  do {                                                                      \
+    if (tstatus.statusCode != hs2::TStatusCode::SUCCESS_STATUS &&           \
+        tstatus.statusCode != hs2::TStatusCode::SUCCESS_WITH_INFO_STATUS) { \
+      return TStatusToStatus(tstatus);                                      \
+    }                                                                       \
+  } while (0)
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/.gitignore b/cpp/src/arrow/dbi/hiveserver2/thrift/.gitignore
new file mode 100644
index 0000000..f510e7c
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/.gitignore
@@ -0,0 +1 @@
+ErrorCodes.thrift
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/CMakeLists.txt b/cpp/src/arrow/dbi/hiveserver2/thrift/CMakeLists.txt
new file mode 100644
index 0000000..c59fd5a
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/CMakeLists.txt
@@ -0,0 +1,110 @@
+# Copyright 2012 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.
+
+# Helper function to generate build rules.  For each input thrift file, this function will
+# generate a rule that maps the input file to the output c++ file.
+# Thrift will generate multiple output files for each input (including java files) and
+# ideally, we'd specify all of the outputs for dependency tracking.
+# Unfortunately, it's not easy to figure out all the output files without parsing the
+# thrift input. (TODO: can thrift tells us what the java output files will be?)
+# The list of output files is used for build dependency tracking so it's not necessary to
+# capture all the output files.
+#
+# To call this function, pass it the output file list followed by the input thrift files:
+#    i.e. HS2_THRIFT_GEN(OUTPUT_FILES, ${THRIFT_FILES})
+#
+# cmake seems to be case sensitive for some keywords. Changing the first IF check to lower
+# case makes it not work.  TODO: investigate this
+function(HS2_THRIFT_GEN VAR)
+  IF (NOT ARGN)
+    MESSAGE(SEND_ERROR "Error: THRIFT_GEN called without any src files")
+    RETURN()
+  ENDIF(NOT ARGN)
+
+  set(${VAR})
+  foreach(FIL ${ARGN})
+    # Get full path
+    get_filename_component(ABS_FIL ${FIL} ABSOLUTE)
+    # Get basename
+    get_filename_component(FIL_WE ${FIL} NAME_WE)
+
+    set(GEN_DIR "${OUTPUT_DIR}/arrow/dbi/hiveserver2")
+
+    # All the output files we can determine based on filename.
+    #   - Does not include .skeleton.cpp files
+    #   - Does not include java output files
+    set(OUTPUT_BE_FILE "${GEN_DIR}/${FIL_WE}_types.cpp")
+    set(OUTPUT_BE_FILE ${OUTPUT_BE_FILE} " ${GEN_DIR}/${FIL_WE}_types.h")
+    set(OUTPUT_BE_FILE ${OUTPUT_BE_FILE} " ${GEN_DIR}/${FIL_WE}_constants.cpp")
+    set(OUTPUT_BE_FILE ${OUTPUT_BE_FILE} " ${GEN_DIR}/${FIL_WE}_constants.h")
+    list(APPEND ${VAR} ${OUTPUT_BE_FILE})
+
+    # BeeswaxService thrift generation
+    # It depends on hive_meta_store, which in turn depends on fb303.
+    # The java dependency is handled by maven.
+    # We need to generate C++ src file for the parent dependencies using the "-r" option.
+    set(CPP_ARGS -nowarn --gen cpp -out ${GEN_DIR})
+    IF (FIL STREQUAL "beeswax.thrift")
+      set(CPP_ARGS -r -nowarn --gen cpp -out ${GEN_DIR})
+    ENDIF(FIL STREQUAL "beeswax.thrift")
+
+    # Be able to include generated ErrorCodes.thrift file
+    set(CPP_ARGS ${CPP_ARGS} -I ${CMAKE_CURRENT_BINARY_DIR})
+
+    add_custom_command(
+      OUTPUT ${OUTPUT_BE_FILE}
+      COMMAND ${THRIFT_COMPILER} ${CPP_ARGS} ${FIL}
+      DEPENDS ${ABS_FIL}
+      COMMENT "Running thrift compiler on ${FIL}"
+	  WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}
+      VERBATIM
+    )
+  endforeach(FIL)
+
+  set(${VAR} ${${VAR}} PARENT_SCOPE)
+endfunction(HS2_THRIFT_GEN)
+
+message("Using Thrift compiler: ${THRIFT_COMPILER}")
+
+set(OUTPUT_DIR ${CMAKE_BINARY_DIR}/src)
+file(MAKE_DIRECTORY ${OUTPUT_DIR})
+
+add_custom_command(OUTPUT ${CMAKE_CURRENT_BINARY_DIR}/ErrorCodes.thrift
+  COMMAND python generate_error_codes.py ${CMAKE_CURRENT_BINARY_DIR}
+  DEPENDS generate_error_codes.py
+  WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR})
+
+set(SRC_FILES
+  ${CMAKE_CURRENT_BINARY_DIR}/ErrorCodes.thrift
+  beeswax.thrift
+  TCLIService.thrift
+  ExecStats.thrift
+  ImpalaService.thrift
+  Status.thrift
+  Types.thrift
+)
+
+SET_SOURCE_FILES_PROPERTIES(Status.thrift PROPERTIES OBJECT_DEPENDS
+  ${CMAKE_CURRENT_BINARY_DIR}/ErrorCodes.thrift)
+
+# Create a build command for each of the thrift src files and generate
+# a list of files they produce
+HS2_THRIFT_GEN(THRIFT_ALL_FILES ${SRC_FILES})
+
+# Add a custom target that generates all the thrift files
+add_custom_target(hs2-thrift-cpp ALL DEPENDS ${THRIFT_ALL_FILES})
+
+add_custom_target(hs2-thrift-generated-files-error DEPENDS
+  ${CMAKE_CURRENT_BINARY_DIR}/ErrorCodes.thrift)
+add_dependencies(hs2-thrift-cpp hs2-thrift-generated-files-error)
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/ExecStats.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/ExecStats.thrift
new file mode 100644
index 0000000..bcf5c4c
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/ExecStats.thrift
@@ -0,0 +1,103 @@
+// Copyright 2012 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.
+
+namespace cpp impala
+namespace java com.cloudera.impala.thrift
+
+include "Status.thrift"
+include "Types.thrift"
+
+enum TExecState {
+  REGISTERED = 0,
+  PLANNING = 1,
+  QUEUED = 2,
+  RUNNING = 3,
+  FINISHED = 4,
+
+  CANCELLED = 5,
+  FAILED = 6,
+}
+
+// Execution stats for a single plan node.
+struct TExecStats {
+  // The wall clock time spent on the "main" thread. This is the user perceived
+  // latency. This value indicates the current bottleneck.
+  // Note: anywhere we have a queue between operators, this time can fluctuate
+  // significantly without the overall query time changing much (i.e. the bottleneck
+  // moved to another operator). This is unavoidable though.
+  1: optional i64 latency_ns
+
+  // Total CPU time spent across all threads. For operators that have an async
+  // component (e.g. multi-threaded) this will be >= latency_ns.
+  2: optional i64 cpu_time_ns
+
+  // Number of rows returned.
+  3: optional i64 cardinality
+
+  // Peak memory used (in bytes).
+  4: optional i64 memory_used
+}
+
+// Summary for a single plan node. This includes labels for how to display the
+// node as well as per instance stats.
+struct TPlanNodeExecSummary {
+  1: required Types.TPlanNodeId node_id
+  2: required i32 fragment_id
+  3: required string label
+  4: optional string label_detail
+  5: required i32 num_children
+
+  // Estimated stats generated by the planner
+  6: optional TExecStats estimated_stats
+
+  // One entry for each BE executing this plan node.
+  7: optional list<TExecStats> exec_stats
+
+  // One entry for each BE executing this plan node. True if this plan node is still
+  // running.
+  8: optional list<bool> is_active
+
+  // If true, this plan node is an exchange node that is the receiver of a broadcast.
+  9: optional bool is_broadcast
+}
+
+// Progress counters for an in-flight query.
+struct TExecProgress {
+  1: optional i64 total_scan_ranges
+  2: optional i64 num_completed_scan_ranges
+}
+
+// Execution summary of an entire query.
+struct TExecSummary {
+  // State of the query.
+  1: required TExecState state
+
+  // Contains the error if state is FAILED.
+  2: optional Status.TStatus status
+
+  // Flattened execution summary of the plan tree.
+  3: optional list<TPlanNodeExecSummary> nodes
+
+  // For each exch node in 'nodes', contains the index to the root node of the sending
+  // fragment for this exch. Both the key and value are indices into 'nodes'.
+  4: optional map<i32, i32> exch_to_sender_map
+
+  // List of errors that were encountered during execution. This can be non-empty
+  // even if status is okay, in which case it contains errors that impala skipped
+  // over.
+  5: optional list<string> error_logs
+
+  // Optional record indicating the query progress
+  6: optional TExecProgress progress
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/ImpalaService.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/ImpalaService.thrift
new file mode 100644
index 0000000..33f049f
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/ImpalaService.thrift
@@ -0,0 +1,300 @@
+// Copyright 2012 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.
+
+namespace cpp impala
+namespace java com.cloudera.impala.thrift
+
+include "ExecStats.thrift"
+include "Status.thrift"
+include "Types.thrift"
+include "beeswax.thrift"
+include "TCLIService.thrift"
+
+// ImpalaService accepts query execution options through beeswax.Query.configuration in
+// key:value form. For example, the list of strings could be:
+//     "num_nodes:1", "abort_on_error:false"
+// The valid keys are listed in this enum. They map to TQueryOptions.
+// Note: If you add an option or change the default, you also need to update:
+// - ImpalaInternalService.thrift: TQueryOptions
+// - SetQueryOption(), SetQueryOptions()
+// - TQueryOptionsToMap()
+enum TImpalaQueryOptions {
+  // if true, abort execution on the first error
+  ABORT_ON_ERROR,
+
+  // maximum # of errors to be reported; Unspecified or 0 indicates backend default
+  MAX_ERRORS,
+
+  // if true, disable llvm codegen
+  DISABLE_CODEGEN,
+
+  // batch size to be used by backend; Unspecified or a size of 0 indicates backend
+  // default
+  BATCH_SIZE,
+
+  // a per-machine approximate limit on the memory consumption of this query;
+  // unspecified or a limit of 0 means no limit;
+  // otherwise specified either as:
+  // a) an int (= number of bytes);
+  // b) a float followed by "M" (MB) or "G" (GB)
+  MEM_LIMIT,
+
+  // specifies the degree of parallelism with which to execute the query;
+  // 1: single-node execution
+  // NUM_NODES_ALL: executes on all nodes that contain relevant data
+  // NUM_NODES_ALL_RACKS: executes on one node per rack that holds relevant data
+  // > 1: executes on at most that many nodes at any point in time (ie, there can be
+  //      more nodes than numNodes with plan fragments for this query, but at most
+  //      numNodes would be active at any point in time)
+  // Constants (NUM_NODES_ALL, NUM_NODES_ALL_RACKS) are defined in JavaConstants.thrift.
+  NUM_NODES,
+
+  // maximum length of the scan range; only applicable to HDFS scan range; Unspecified or
+  // a length of 0 indicates backend default;
+  MAX_SCAN_RANGE_LENGTH,
+
+  // Maximum number of io buffers (per disk)
+  MAX_IO_BUFFERS,
+
+  // Number of scanner threads.
+  NUM_SCANNER_THREADS,
+
+  // If true, Impala will try to execute on file formats that are not fully supported yet
+  ALLOW_UNSUPPORTED_FORMATS,
+
+  // if set and > -1, specifies the default limit applied to a top-level SELECT statement
+  // with an ORDER BY but without a LIMIT clause (ie, if the SELECT statement also has
+  // a LIMIT clause, this default is ignored)
+  DEFAULT_ORDER_BY_LIMIT,
+
+  // DEBUG ONLY:
+  // If set to
+  //   "[<backend number>:]<node id>:<TExecNodePhase>:<TDebugAction>",
+  // the exec node with the given id will perform the specified action in the given
+  // phase. If the optional backend number (starting from 0) is specified, only that
+  // backend instance will perform the debug action, otherwise all backends will behave
+  // in that way.
+  // If the string doesn't have the required format or if any of its components is
+  // invalid, the option is ignored.
+  DEBUG_ACTION,
+
+  // If true, raise an error when the DEFAULT_ORDER_BY_LIMIT has been reached.
+  ABORT_ON_DEFAULT_LIMIT_EXCEEDED,
+
+  // Compression codec when inserting into tables.
+  // Valid values are "snappy", "gzip", "bzip2" and "none"
+  // Leave blank to use default.
+  COMPRESSION_CODEC,
+
+  // Mode for compressing sequence files; either BLOCK, RECORD, or DEFAULT
+  SEQ_COMPRESSION_MODE,
+
+  // HBase scan query option. If set and > 0, HBASE_CACHING is the value for
+  // "hbase.client.Scan.setCaching()" when querying HBase table. Otherwise, use backend
+  // default.
+  // If the value is too high, then the hbase region server will have a hard time (GC
+  // pressure and long response times). If the value is too small, then there will be
+  // extra trips to the hbase region server.
+  HBASE_CACHING,
+
+  // HBase scan query option. If set, HBase scan will always set
+  // "hbase.client.setCacheBlocks" to CACHE_BLOCKS. Default is false.
+  // If the table is large and the query is doing big scan, set it to false to
+  // avoid polluting the cache in the hbase region server.
+  // If the table is small and the table is used several time, set it to true to improve
+  // performance.
+  HBASE_CACHE_BLOCKS,
+
+  // Target file size for inserts into parquet tables. 0 uses the default.
+  PARQUET_FILE_SIZE,
+
+  // Level of detail for explain output (NORMAL, VERBOSE).
+  EXPLAIN_LEVEL,
+
+  // If true, waits for the result of all catalog operations to be processed by all
+  // active impalad in the cluster before completing.
+  SYNC_DDL,
+
+  // Request pool this request should be submitted to. If not set
+  // the pool is determined based on the user.
+  REQUEST_POOL,
+
+  // Per-host virtual CPU cores required for query (only relevant with RM).
+  V_CPU_CORES,
+
+  // Max time in milliseconds the resource broker should wait for
+  // a resource request to be granted by Llama/Yarn (only relevant with RM).
+  RESERVATION_REQUEST_TIMEOUT,
+
+  // if true, disables cached reads. This option has no effect if REPLICA_PREFERENCE is
+  // configured.
+  // TODO: Retire in C6
+  DISABLE_CACHED_READS,
+
+  // Temporary testing flag
+  DISABLE_OUTERMOST_TOPN,
+
+  // Size of initial memory reservation when RM is enabled
+  RM_INITIAL_MEM,
+
+  // Time, in s, before a query will be timed out if it is inactive. May not exceed
+  // --idle_query_timeout if that flag > 0.
+  QUERY_TIMEOUT_S,
+
+  // Test hook for spill to disk operators
+  MAX_BLOCK_MGR_MEMORY,
+
+  // Transforms all count(distinct) aggregations into NDV()
+  APPX_COUNT_DISTINCT,
+
+  // If true, allows Impala to internally disable spilling for potentially
+  // disastrous query plans. Impala will excercise this option if a query
+  // has no plan hints, and at least one table is missing relevant stats.
+  DISABLE_UNSAFE_SPILLS,
+
+  // If the number of rows that are processed for a single query is below the
+  // threshold, it will be executed on the coordinator only with codegen disabled
+  EXEC_SINGLE_NODE_ROWS_THRESHOLD,
+
+  // If true, use the table's metadata to produce the partition columns instead of table
+  // scans whenever possible. This option is opt-in by default as this optimization may
+  // produce different results than the scan based approach in some edge cases.
+  OPTIMIZE_PARTITION_KEY_SCANS,
+
+  // Prefered memory distance of replicas. This parameter determines the pool of replicas
+  // among which scans will be scheduled in terms of the distance of the replica storage
+  // from the impalad.
+  REPLICA_PREFERENCE,
+
+  // Determines tie breaking policy when picking locations.
+  RANDOM_REPLICA,
+
+  // For scan nodes with any conjuncts, use codegen to evaluate the conjuncts if
+  // the number of rows * number of operators in the conjuncts exceeds this threshold.
+  SCAN_NODE_CODEGEN_THRESHOLD,
+
+  // If true, the planner will not generate plans with streaming preaggregations.
+  DISABLE_STREAMING_PREAGGREGATIONS,
+
+  RUNTIME_FILTER_MODE,
+
+  // Size (in bytes) of a runtime Bloom Filter. Will be rounded up to nearest power of
+  // two.
+  RUNTIME_BLOOM_FILTER_SIZE,
+
+  // Time (in ms) to wait in scans for partition filters to arrive.
+  RUNTIME_FILTER_WAIT_TIME_MS,
+
+  // If true, disable application of runtime filters to individual rows.
+  DISABLE_ROW_RUNTIME_FILTERING,
+
+  // Maximum number of runtime filters allowed per query.
+  MAX_NUM_RUNTIME_FILTERS
+}
+
+// The summary of an insert.
+struct TInsertResult {
+  // Number of appended rows per modified partition. Only applies to HDFS tables.
+  // The keys represent partitions to create, coded as k1=v1/k2=v2/k3=v3..., with the
+  // root in an unpartitioned table being the empty string.
+  1: required map<string, i64> rows_appended
+}
+
+// Response from a call to PingImpalaService
+struct TPingImpalaServiceResp {
+  // The Impala service's version string.
+  1: string version
+}
+
+// Parameters for a ResetTable request which will invalidate a table's metadata.
+// DEPRECATED.
+struct TResetTableReq {
+  // Name of the table's parent database.
+  1: required string db_name
+
+  // Name of the table.
+  2: required string table_name
+}
+
+// For all rpc that return a TStatus as part of their result type,
+// if the status_code field is set to anything other than OK, the contents
+// of the remainder of the result type is undefined (typically not set)
+service ImpalaService extends beeswax.BeeswaxService {
+  // Cancel execution of query. Returns RUNTIME_ERROR if query_id
+  // unknown.
+  // This terminates all threads running on behalf of this query at
+  // all nodes that were involved in the execution.
+  // Throws BeeswaxException if the query handle is invalid (this doesn't
+  // necessarily indicate an error: the query might have finished).
+  Status.TStatus Cancel(1:beeswax.QueryHandle query_id)
+      throws(1:beeswax.BeeswaxException error);
+
+  // Invalidates all catalog metadata, forcing a reload
+  // DEPRECATED; execute query "invalidate metadata" to refresh metadata
+  Status.TStatus ResetCatalog();
+
+  // Invalidates a specific table's catalog metadata, forcing a reload on the next access
+  // DEPRECATED; execute query "refresh <table>" to refresh metadata
+  Status.TStatus ResetTable(1:TResetTableReq request)
+
+  // Returns the runtime profile string for the given query handle.
+  string GetRuntimeProfile(1:beeswax.QueryHandle query_id)
+      throws(1:beeswax.BeeswaxException error);
+
+  // Closes the query handle and return the result summary of the insert.
+  TInsertResult CloseInsert(1:beeswax.QueryHandle handle)
+      throws(1:beeswax.QueryNotFoundException error, 2:beeswax.BeeswaxException error2);
+
+  // Client calls this RPC to verify that the server is an ImpalaService. Returns the
+  // server version.
+  TPingImpalaServiceResp PingImpalaService();
+
+  // Returns the summary of the current execution.
+  ExecStats.TExecSummary GetExecSummary(1:beeswax.QueryHandle handle)
+      throws(1:beeswax.QueryNotFoundException error, 2:beeswax.BeeswaxException error2);
+}
+
+// Impala HiveServer2 service
+
+struct TGetExecSummaryReq {
+  1: optional TCLIService.TOperationHandle operationHandle
+
+  2: optional TCLIService.TSessionHandle sessionHandle
+}
+
+struct TGetExecSummaryResp {
+  1: required TCLIService.TStatus status
+
+  2: optional ExecStats.TExecSummary summary
+}
+
+struct TGetRuntimeProfileReq {
+  1: optional TCLIService.TOperationHandle operationHandle
+
+  2: optional TCLIService.TSessionHandle sessionHandle
+}
+
+struct TGetRuntimeProfileResp {
+  1: required TCLIService.TStatus status
+
+  2: optional string profile
+}
+
+service ImpalaHiveServer2Service extends TCLIService.TCLIService {
+  // Returns the exec summary for the given query
+  TGetExecSummaryResp GetExecSummary(1:TGetExecSummaryReq req);
+
+  // Returns the runtime profile string for the given query
+  TGetRuntimeProfileResp GetRuntimeProfile(1:TGetRuntimeProfileReq req);
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/Status.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/Status.thrift
new file mode 100644
index 0000000..db9518e
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/Status.thrift
@@ -0,0 +1,23 @@
+// Copyright 2012 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.
+
+include "ErrorCodes.thrift"
+
+namespace cpp impala
+namespace java com.cloudera.impala.thrift
+
+struct TStatus {
+  1: required ErrorCodes.TErrorCode status_code
+  2: list<string> error_msgs
+}
\ No newline at end of file
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/TCLIService.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/TCLIService.thrift
new file mode 100644
index 0000000..f95e2f8
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/TCLIService.thrift
@@ -0,0 +1,1180 @@
+// 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.
+
+// Coding Conventions for this file:
+//
+// Structs/Enums/Unions
+// * Struct, Enum, and Union names begin with a "T",
+//   and use a capital letter for each new word, with no underscores.
+// * All fields should be declared as either optional or required.
+//
+// Functions
+// * Function names start with a capital letter and have a capital letter for
+//   each new word, with no underscores.
+// * Each function should take exactly one parameter, named TFunctionNameReq,
+//   and should return either void or TFunctionNameResp. This convention allows
+//   incremental updates.
+//
+// Services
+// * Service names begin with the letter "T", use a capital letter for each
+//   new word (with no underscores), and end with the word "Service".
+
+namespace java org.apache.hive.service.cli.thrift
+namespace cpp apache.hive.service.cli.thrift
+
+// List of protocol versions. A new token should be
+// added to the end of this list every time a change is made.
+enum TProtocolVersion {
+  HIVE_CLI_SERVICE_PROTOCOL_V1,
+
+  // V2 adds support for asynchronous execution
+  HIVE_CLI_SERVICE_PROTOCOL_V2
+
+  // V3 add varchar type, primitive type qualifiers
+  HIVE_CLI_SERVICE_PROTOCOL_V3
+
+  // V4 add decimal precision/scale, char type
+  HIVE_CLI_SERVICE_PROTOCOL_V4
+
+  // V5 adds error details when GetOperationStatus returns in error state
+  HIVE_CLI_SERVICE_PROTOCOL_V5
+
+  // V6 uses binary type for binary payload (was string) and uses columnar result set
+  HIVE_CLI_SERVICE_PROTOCOL_V6
+
+  // V7 adds support for delegation token based connection
+  HIVE_CLI_SERVICE_PROTOCOL_V7
+}
+
+enum TTypeId {
+  BOOLEAN_TYPE,
+  TINYINT_TYPE,
+  SMALLINT_TYPE,
+  INT_TYPE,
+  BIGINT_TYPE,
+  FLOAT_TYPE,
+  DOUBLE_TYPE,
+  STRING_TYPE,
+  TIMESTAMP_TYPE,
+  BINARY_TYPE,
+  ARRAY_TYPE,
+  MAP_TYPE,
+  STRUCT_TYPE,
+  UNION_TYPE,
+  USER_DEFINED_TYPE,
+  DECIMAL_TYPE,
+  NULL_TYPE,
+  DATE_TYPE,
+  VARCHAR_TYPE,
+  CHAR_TYPE
+}
+
+const set<TTypeId> PRIMITIVE_TYPES = [
+  TTypeId.BOOLEAN_TYPE,
+  TTypeId.TINYINT_TYPE,
+  TTypeId.SMALLINT_TYPE,
+  TTypeId.INT_TYPE,
+  TTypeId.BIGINT_TYPE,
+  TTypeId.FLOAT_TYPE,
+  TTypeId.DOUBLE_TYPE,
+  TTypeId.STRING_TYPE,
+  TTypeId.TIMESTAMP_TYPE,
+  TTypeId.BINARY_TYPE,
+  TTypeId.DECIMAL_TYPE,
+  TTypeId.NULL_TYPE,
+  TTypeId.DATE_TYPE,
+  TTypeId.VARCHAR_TYPE,
+  TTypeId.CHAR_TYPE
+]
+
+const set<TTypeId> COMPLEX_TYPES = [
+  TTypeId.ARRAY_TYPE
+  TTypeId.MAP_TYPE
+  TTypeId.STRUCT_TYPE
+  TTypeId.UNION_TYPE
+  TTypeId.USER_DEFINED_TYPE
+]
+
+const set<TTypeId> COLLECTION_TYPES = [
+  TTypeId.ARRAY_TYPE
+  TTypeId.MAP_TYPE
+]
+
+const map<TTypeId,string> TYPE_NAMES = {
+  TTypeId.BOOLEAN_TYPE: "BOOLEAN",
+  TTypeId.TINYINT_TYPE: "TINYINT",
+  TTypeId.SMALLINT_TYPE: "SMALLINT",
+  TTypeId.INT_TYPE: "INT",
+  TTypeId.BIGINT_TYPE: "BIGINT",
+  TTypeId.FLOAT_TYPE: "FLOAT",
+  TTypeId.DOUBLE_TYPE: "DOUBLE",
+  TTypeId.STRING_TYPE: "STRING",
+  TTypeId.TIMESTAMP_TYPE: "TIMESTAMP",
+  TTypeId.BINARY_TYPE: "BINARY",
+  TTypeId.ARRAY_TYPE: "ARRAY",
+  TTypeId.MAP_TYPE: "MAP",
+  TTypeId.STRUCT_TYPE: "STRUCT",
+  TTypeId.UNION_TYPE: "UNIONTYPE",
+  TTypeId.DECIMAL_TYPE: "DECIMAL",
+  TTypeId.NULL_TYPE: "NULL"
+  TTypeId.DATE_TYPE: "DATE"
+  TTypeId.VARCHAR_TYPE: "VARCHAR"
+  TTypeId.CHAR_TYPE: "CHAR"
+}
+
+// Thrift does not support recursively defined types or forward declarations,
+// which makes it difficult to represent Hive's nested types.
+// To get around these limitations TTypeDesc employs a type list that maps
+// integer "pointers" to TTypeEntry objects. The following examples show
+// how different types are represented using this scheme:
+//
+// "INT":
+// TTypeDesc {
+//   types = [
+//     TTypeEntry.primitive_entry {
+//       type = INT_TYPE
+//     }
+//   ]
+// }
+//
+// "ARRAY<INT>":
+// TTypeDesc {
+//   types = [
+//     TTypeEntry.array_entry {
+//       object_type_ptr = 1
+//     },
+//     TTypeEntry.primitive_entry {
+//       type = INT_TYPE
+//     }
+//   ]
+// }
+//
+// "MAP<INT,STRING>":
+// TTypeDesc {
+//   types = [
+//     TTypeEntry.map_entry {
+//       key_type_ptr = 1
+//       value_type_ptr = 2
+//     },
+//     TTypeEntry.primitive_entry {
+//       type = INT_TYPE
+//     },
+//     TTypeEntry.primitive_entry {
+//       type = STRING_TYPE
+//     }
+//   ]
+// }
+
+typedef i32 TTypeEntryPtr
+
+// Valid TTypeQualifiers key names
+const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"
+
+// Type qualifier key name for decimal
+const string PRECISION = "precision"
+const string SCALE = "scale"
+
+union TTypeQualifierValue {
+  1: optional i32 i32Value
+  2: optional string stringValue
+}
+
+// Type qualifiers for primitive type.
+struct TTypeQualifiers {
+  1: required map <string, TTypeQualifierValue> qualifiers
+}
+
+// Type entry for a primitive type.
+struct TPrimitiveTypeEntry {
+  // The primitive type token. This must satisfy the condition
+  // that type is in the PRIMITIVE_TYPES set.
+  1: required TTypeId type
+  2: optional TTypeQualifiers typeQualifiers
+}
+
+// Type entry for an ARRAY type.
+struct TArrayTypeEntry {
+  1: required TTypeEntryPtr objectTypePtr
+}
+
+// Type entry for a MAP type.
+struct TMapTypeEntry {
+  1: required TTypeEntryPtr keyTypePtr
+  2: required TTypeEntryPtr valueTypePtr
+}
+
+// Type entry for a STRUCT type.
+struct TStructTypeEntry {
+  1: required map<string, TTypeEntryPtr> nameToTypePtr
+}
+
+// Type entry for a UNIONTYPE type.
+struct TUnionTypeEntry {
+  1: required map<string, TTypeEntryPtr> nameToTypePtr
+}
+
+struct TUserDefinedTypeEntry {
+  // The fully qualified name of the class implementing this type.
+  1: required string typeClassName
+}
+
+// We use a union here since Thrift does not support inheritance.
+union TTypeEntry {
+  1: TPrimitiveTypeEntry primitiveEntry
+  2: TArrayTypeEntry arrayEntry
+  3: TMapTypeEntry mapEntry
+  4: TStructTypeEntry structEntry
+  5: TUnionTypeEntry unionEntry
+  6: TUserDefinedTypeEntry userDefinedTypeEntry
+}
+
+// Type descriptor for columns.
+struct TTypeDesc {
+  // The "top" type is always the first element of the list.
+  // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE
+  // type, then subsequent elements represent nested types.
+  1: required list<TTypeEntry> types
+}
+
+// A result set column descriptor.
+struct TColumnDesc {
+  // The name of the column
+  1: required string columnName
+
+  // The type descriptor for this column
+  2: required TTypeDesc typeDesc
+
+  // The ordinal position of this column in the schema
+  3: required i32 position
+
+  4: optional string comment
+}
+
+// Metadata used to describe the schema (column names, types, comments)
+// of result sets.
+struct TTableSchema {
+  1: required list<TColumnDesc> columns
+}
+
+// A Boolean column value.
+struct TBoolValue {
+  // NULL if value is unset.
+  1: optional bool value
+}
+
+// A Byte column value.
+struct TByteValue {
+  // NULL if value is unset.
+  1: optional byte value
+}
+
+// A signed, 16 bit column value.
+struct TI16Value {
+  // NULL if value is unset
+  1: optional i16 value
+}
+
+// A signed, 32 bit column value
+struct TI32Value {
+  // NULL if value is unset
+  1: optional i32 value
+}
+
+// A signed 64 bit column value
+struct TI64Value {
+  // NULL if value is unset
+  1: optional i64 value
+}
+
+// A floating point 64 bit column value
+struct TDoubleValue {
+  // NULL if value is unset
+  1: optional double value
+}
+
+struct TStringValue {
+  // NULL if value is unset
+  1: optional string value
+}
+
+// A single column value in a result set.
+// Note that Hive's type system is richer than Thrift's,
+// so in some cases we have to map multiple Hive types
+// to the same Thrift type. On the client-side this is
+// disambiguated by looking at the Schema of the
+// result set.
+union TColumnValue {
+  1: TBoolValue   boolVal      // BOOLEAN
+  2: TByteValue   byteVal      // TINYINT
+  3: TI16Value    i16Val       // SMALLINT
+  4: TI32Value    i32Val       // INT
+  5: TI64Value    i64Val       // BIGINT, TIMESTAMP
+  6: TDoubleValue doubleVal    // FLOAT, DOUBLE
+  7: TStringValue stringVal    // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL
+}
+
+// Represents a row in a rowset.
+struct TRow {
+  1: required list<TColumnValue> colVals
+}
+
+struct TBoolColumn {
+  1: required list<bool> values
+  2: required binary nulls
+}
+
+struct TByteColumn {
+  1: required list<byte> values
+  2: required binary nulls
+}
+
+struct TI16Column {
+  1: required list<i16> values
+  2: required binary nulls
+}
+
+struct TI32Column {
+  1: required list<i32> values
+  2: required binary nulls
+}
+
+struct TI64Column {
+  1: required list<i64> values
+  2: required binary nulls
+}
+
+struct TDoubleColumn {
+  1: required list<double> values
+  2: required binary nulls
+}
+
+struct TStringColumn {
+  1: required list<string> values
+  2: required binary nulls
+}
+
+struct TBinaryColumn {
+  1: required list<binary> values
+  2: required binary nulls
+}
+
+// Note that Hive's type system is richer than Thrift's,
+// so in some cases we have to map multiple Hive types
+// to the same Thrift type. On the client-side this is
+// disambiguated by looking at the Schema of the
+// result set.
+union TColumn {
+  1: TBoolColumn   boolVal      // BOOLEAN
+  2: TByteColumn   byteVal      // TINYINT
+  3: TI16Column    i16Val       // SMALLINT
+  4: TI32Column    i32Val       // INT
+  5: TI64Column    i64Val       // BIGINT, TIMESTAMP
+  6: TDoubleColumn doubleVal    // FLOAT, DOUBLE
+  7: TStringColumn stringVal    // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL
+  8: TBinaryColumn binaryVal    // BINARY
+}
+
+// Represents a rowset
+struct TRowSet {
+  // The starting row offset of this rowset.
+  1: required i64 startRowOffset
+  2: required list<TRow> rows
+  3: optional list<TColumn> columns
+}
+
+// The return status code contained in each response.
+enum TStatusCode {
+  SUCCESS_STATUS,
+  SUCCESS_WITH_INFO_STATUS,
+  STILL_EXECUTING_STATUS,
+  ERROR_STATUS,
+  INVALID_HANDLE_STATUS
+}
+
+// The return status of a remote request
+struct TStatus {
+  1: required TStatusCode statusCode
+
+  // If status is SUCCESS_WITH_INFO, info_msgs may be populated with
+  // additional diagnostic information.
+  2: optional list<string> infoMessages
+
+  // If status is ERROR, then the following fields may be set
+  3: optional string sqlState  // as defined in the ISO/IEF CLI specification
+  4: optional i32 errorCode    // internal error code
+  5: optional string errorMessage
+}
+
+// The state of an operation (i.e. a query or other
+// asynchronous operation that generates a result set)
+// on the server.
+enum TOperationState {
+  // The operation has been initialized
+  INITIALIZED_STATE,
+
+  // The operation is running. In this state the result
+  // set is not available.
+  RUNNING_STATE,
+
+  // The operation has completed. When an operation is in
+  // this state its result set may be fetched.
+  FINISHED_STATE,
+
+  // The operation was canceled by a client
+  CANCELED_STATE,
+
+  // The operation was closed by a client
+  CLOSED_STATE,
+
+  // The operation failed due to an error
+  ERROR_STATE,
+
+  // The operation is in an unrecognized state
+  UKNOWN_STATE,
+
+  // The operation is in an pending state
+  PENDING_STATE,
+}
+
+// A string identifier. This is interpreted literally.
+typedef string TIdentifier
+
+// A search pattern.
+//
+// Valid search pattern characters:
+// '_': Any single character.
+// '%': Any sequence of zero or more characters.
+// '\': Escape character used to include special characters,
+//      e.g. '_', '%', '\'. If a '\' precedes a non-special
+//      character it has no special meaning and is interpreted
+//      literally.
+typedef string TPattern
+
+
+// A search pattern or identifier. Used as input
+// parameter for many of the catalog functions.
+typedef string TPatternOrIdentifier
+
+struct THandleIdentifier {
+  // 16 byte globally unique identifier
+  // This is the public ID of the handle and
+  // can be used for reporting.
+  1: required binary guid,
+
+  // 16 byte secret generated by the server
+  // and used to verify that the handle is not
+  // being hijacked by another user.
+  2: required binary secret,
+}
+
+// Client-side handle to persistent
+// session information on the server-side.
+struct TSessionHandle {
+  1: required THandleIdentifier sessionId
+}
+
+// The subtype of an OperationHandle.
+enum TOperationType {
+  EXECUTE_STATEMENT,
+  GET_TYPE_INFO,
+  GET_CATALOGS,
+  GET_SCHEMAS,
+  GET_TABLES,
+  GET_TABLE_TYPES,
+  GET_COLUMNS,
+  GET_FUNCTIONS,
+  UNKNOWN,
+}
+
+// Client-side reference to a task running
+// asynchronously on the server.
+struct TOperationHandle {
+  1: required THandleIdentifier operationId
+  2: required TOperationType operationType
+
+  // If hasResultSet = TRUE, then this operation
+  // generates a result set that can be fetched.
+  // Note that the result set may be empty.
+  //
+  // If hasResultSet = FALSE, then this operation
+  // does not generate a result set, and calling
+  // GetResultSetMetadata or FetchResults against
+  // this OperationHandle will generate an error.
+  3: required bool hasResultSet
+
+  // For operations that don't generate result sets,
+  // modifiedRowCount is either:
+  //
+  // 1) The number of rows that were modified by
+  //    the DML operation (e.g. number of rows inserted,
+  //    number of rows deleted, etc).
+  //
+  // 2) 0 for operations that don't modify or add rows.
+  //
+  // 3) < 0 if the operation is capable of modifiying rows,
+  //    but Hive is unable to determine how many rows were
+  //    modified. For example, Hive's LOAD DATA command
+  //    doesn't generate row count information because
+  //    Hive doesn't inspect the data as it is loaded.
+  //
+  // modifiedRowCount is unset if the operation generates
+  // a result set.
+  4: optional double modifiedRowCount
+}
+
+
+// OpenSession()
+//
+// Open a session (connection) on the server against
+// which operations may be executed.
+struct TOpenSessionReq {
+  // The version of the HiveServer2 protocol that the client is using.
+  1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6
+
+  // Username and password for authentication.
+  // Depending on the authentication scheme being used,
+  // this information may instead be provided by a lower
+  // protocol layer, in which case these fields may be
+  // left unset.
+  2: optional string username
+  3: optional string password
+
+  // Configuration overlay which is applied when the session is
+  // first created.
+  4: optional map<string, string> configuration
+}
+
+struct TOpenSessionResp {
+  1: required TStatus status
+
+  // The protocol version that the server is using.
+  2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6
+
+  // Session Handle
+  3: optional TSessionHandle sessionHandle
+
+  // The configuration settings for this session.
+  4: optional map<string, string> configuration
+}
+
+
+// CloseSession()
+//
+// Closes the specified session and frees any resources
+// currently allocated to that session. Any open
+// operations in that session will be canceled.
+struct TCloseSessionReq {
+  1: required TSessionHandle sessionHandle
+}
+
+struct TCloseSessionResp {
+  1: required TStatus status
+}
+
+
+
+enum TGetInfoType {
+  CLI_MAX_DRIVER_CONNECTIONS =           0,
+  CLI_MAX_CONCURRENT_ACTIVITIES =        1,
+  CLI_DATA_SOURCE_NAME =                 2,
+  CLI_FETCH_DIRECTION =                  8,
+  CLI_SERVER_NAME =                      13,
+  CLI_SEARCH_PATTERN_ESCAPE =            14,
+  CLI_DBMS_NAME =                        17,
+  CLI_DBMS_VER =                         18,
+  CLI_ACCESSIBLE_TABLES =                19,
+  CLI_ACCESSIBLE_PROCEDURES =            20,
+  CLI_CURSOR_COMMIT_BEHAVIOR =           23,
+  CLI_DATA_SOURCE_READ_ONLY =            25,
+  CLI_DEFAULT_TXN_ISOLATION =            26,
+  CLI_IDENTIFIER_CASE =                  28,
+  CLI_IDENTIFIER_QUOTE_CHAR =            29,
+  CLI_MAX_COLUMN_NAME_LEN =              30,
+  CLI_MAX_CURSOR_NAME_LEN =              31,
+  CLI_MAX_SCHEMA_NAME_LEN =              32,
+  CLI_MAX_CATALOG_NAME_LEN =             34,
+  CLI_MAX_TABLE_NAME_LEN =               35,
+  CLI_SCROLL_CONCURRENCY =               43,
+  CLI_TXN_CAPABLE =                      46,
+  CLI_USER_NAME =                        47,
+  CLI_TXN_ISOLATION_OPTION =             72,
+  CLI_INTEGRITY =                        73,
+  CLI_GETDATA_EXTENSIONS =               81,
+  CLI_NULL_COLLATION =                   85,
+  CLI_ALTER_TABLE =                      86,
+  CLI_ORDER_BY_COLUMNS_IN_SELECT =       90,
+  CLI_SPECIAL_CHARACTERS =               94,
+  CLI_MAX_COLUMNS_IN_GROUP_BY =          97,
+  CLI_MAX_COLUMNS_IN_INDEX =             98,
+  CLI_MAX_COLUMNS_IN_ORDER_BY =          99,
+  CLI_MAX_COLUMNS_IN_SELECT =            100,
+  CLI_MAX_COLUMNS_IN_TABLE =             101,
+  CLI_MAX_INDEX_SIZE =                   102,
+  CLI_MAX_ROW_SIZE =                     104,
+  CLI_MAX_STATEMENT_LEN =                105,
+  CLI_MAX_TABLES_IN_SELECT =             106,
+  CLI_MAX_USER_NAME_LEN =                107,
+  CLI_OJ_CAPABILITIES =                  115,
+
+  CLI_XOPEN_CLI_YEAR =                   10000,
+  CLI_CURSOR_SENSITIVITY =               10001,
+  CLI_DESCRIBE_PARAMETER =               10002,
+  CLI_CATALOG_NAME =                     10003,
+  CLI_COLLATION_SEQ =                    10004,
+  CLI_MAX_IDENTIFIER_LEN =               10005,
+}
+
+union TGetInfoValue {
+  1: string stringValue
+  2: i16 smallIntValue
+  3: i32 integerBitmask
+  4: i32 integerFlag
+  5: i32 binaryValue
+  6: i64 lenValue
+}
+
+// GetInfo()
+//
+// This function is based on ODBC's CLIGetInfo() function.
+// The function returns general information about the data source
+// using the same keys as ODBC.
+struct TGetInfoReq {
+  // The sesssion to run this request against
+  1: required TSessionHandle sessionHandle
+
+  2: required TGetInfoType infoType
+}
+
+struct TGetInfoResp {
+  1: required TStatus status
+
+  2: required TGetInfoValue infoValue
+}
+
+
+// ExecuteStatement()
+//
+// Execute a statement.
+// The returned OperationHandle can be used to check on the
+// status of the statement, and to fetch results once the
+// statement has finished executing.
+struct TExecuteStatementReq {
+  // The session to execute the statement against
+  1: required TSessionHandle sessionHandle
+
+  // The statement to be executed (DML, DDL, SET, etc)
+  2: required string statement
+
+  // Configuration properties that are overlayed on top of the
+  // the existing session configuration before this statement
+  // is executed. These properties apply to this statement
+  // only and will not affect the subsequent state of the Session.
+  3: optional map<string, string> confOverlay
+
+  // Execute asynchronously when runAsync is true
+  4: optional bool runAsync = false
+}
+
+struct TExecuteStatementResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+// GetTypeInfo()
+//
+// Get information about types supported by the HiveServer instance.
+// The information is returned as a result set which can be fetched
+// using the OperationHandle provided in the response.
+//
+// Refer to the documentation for ODBC's CLIGetTypeInfo function for
+// the format of the result set.
+struct TGetTypeInfoReq {
+  // The session to run this request against.
+  1: required TSessionHandle sessionHandle
+}
+
+struct TGetTypeInfoResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetCatalogs()
+//
+// Returns the list of catalogs (databases)
+// Results are ordered by TABLE_CATALOG
+//
+// Resultset columns :
+// col1
+// name: TABLE_CAT
+// type: STRING
+// desc: Catalog name. NULL if not applicable.
+//
+struct TGetCatalogsReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+}
+
+struct TGetCatalogsResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetSchemas()
+//
+// Retrieves the schema names available in this database.
+// The results are ordered by TABLE_CATALOG and TABLE_SCHEM.
+// col1
+// name: TABLE_SCHEM
+// type: STRING
+// desc: schema name
+// col2
+// name: TABLE_CATALOG
+// type: STRING
+// desc: catalog name
+struct TGetSchemasReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+
+  // Name of the catalog. Must not contain a search pattern.
+  2: optional TIdentifier catalogName
+
+  // schema name or pattern
+  3: optional TPatternOrIdentifier schemaName
+}
+
+struct TGetSchemasResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetTables()
+//
+// Returns a list of tables with catalog, schema, and table
+// type information. The information is returned as a result
+// set which can be fetched using the OperationHandle
+// provided in the response.
+// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME
+//
+// Result Set Columns:
+//
+// col1
+// name: TABLE_CAT
+// type: STRING
+// desc: Catalog name. NULL if not applicable.
+//
+// col2
+// name: TABLE_SCHEM
+// type: STRING
+// desc: Schema name.
+//
+// col3
+// name: TABLE_NAME
+// type: STRING
+// desc: Table name.
+//
+// col4
+// name: TABLE_TYPE
+// type: STRING
+// desc: The table type, e.g. "TABLE", "VIEW", etc.
+//
+// col5
+// name: REMARKS
+// type: STRING
+// desc: Comments about the table
+//
+struct TGetTablesReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+
+  // Name of the catalog or a search pattern.
+  2: optional TPatternOrIdentifier catalogName
+
+  // Name of the schema or a search pattern.
+  3: optional TPatternOrIdentifier schemaName
+
+  // Name of the table or a search pattern.
+  4: optional TPatternOrIdentifier tableName
+
+  // List of table types to match
+  // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY",
+  // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc.
+  5: optional list<string> tableTypes
+}
+
+struct TGetTablesResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetTableTypes()
+//
+// Returns the table types available in this database.
+// The results are ordered by table type.
+//
+// col1
+// name: TABLE_TYPE
+// type: STRING
+// desc: Table type name.
+struct TGetTableTypesReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+}
+
+struct TGetTableTypesResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetColumns()
+//
+// Returns a list of columns in the specified tables.
+// The information is returned as a result set which can be fetched
+// using the OperationHandle provided in the response.
+// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME,
+// and ORDINAL_POSITION.
+//
+// Result Set Columns are the same as those for the ODBC CLIColumns
+// function.
+//
+struct TGetColumnsReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+
+  // Name of the catalog. Must not contain a search pattern.
+  2: optional TIdentifier catalogName
+
+  // Schema name or search pattern
+  3: optional TPatternOrIdentifier schemaName
+
+  // Table name or search pattern
+  4: optional TPatternOrIdentifier tableName
+
+  // Column name or search pattern
+  5: optional TPatternOrIdentifier columnName
+}
+
+struct TGetColumnsResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetFunctions()
+//
+// Returns a list of functions supported by the data source. The
+// behavior of this function matches
+// java.sql.DatabaseMetaData.getFunctions() both in terms of
+// inputs and outputs.
+//
+// Result Set Columns:
+//
+// col1
+// name: FUNCTION_CAT
+// type: STRING
+// desc: Function catalog (may be null)
+//
+// col2
+// name: FUNCTION_SCHEM
+// type: STRING
+// desc: Function schema (may be null)
+//
+// col3
+// name: FUNCTION_NAME
+// type: STRING
+// desc: Function name. This is the name used to invoke the function.
+//
+// col4
+// name: REMARKS
+// type: STRING
+// desc: Explanatory comment on the function.
+//
+// col5
+// name: FUNCTION_TYPE
+// type: SMALLINT
+// desc: Kind of function. One of:
+//       * functionResultUnknown - Cannot determine if a return value or a table
+//                                 will be returned.
+//       * functionNoTable       - Does not a return a table.
+//       * functionReturnsTable  - Returns a table.
+//
+// col6
+// name: SPECIFIC_NAME
+// type: STRING
+// desc: The name which uniquely identifies this function within its schema.
+//       In this case this is the fully qualified class name of the class
+//       that implements this function.
+//
+struct TGetFunctionsReq {
+  // Session to run this request against
+  1: required TSessionHandle sessionHandle
+
+  // A catalog name; must match the catalog name as it is stored in the
+  // database; "" retrieves those without a catalog; null means
+  // that the catalog name should not be used to narrow the search.
+  2: optional TIdentifier catalogName
+
+  // A schema name pattern; must match the schema name as it is stored
+  // in the database; "" retrieves those without a schema; null means
+  // that the schema name should not be used to narrow the search.
+  3: optional TPatternOrIdentifier schemaName
+
+  // A function name pattern; must match the function name as it is stored
+  // in the database.
+  4: required TPatternOrIdentifier functionName
+}
+
+struct TGetFunctionsResp {
+  1: required TStatus status
+  2: optional TOperationHandle operationHandle
+}
+
+
+// GetOperationStatus()
+//
+// Get the status of an operation running on the server.
+struct TGetOperationStatusReq {
+  // Session to run this request against
+  1: required TOperationHandle operationHandle
+}
+
+struct TGetOperationStatusResp {
+  1: required TStatus status
+  2: optional TOperationState operationState
+
+  // If operationState is ERROR_STATE, then the following fields may be set
+  // sqlState as defined in the ISO/IEF CLI specification
+  3: optional string sqlState
+
+  // Internal error code
+  4: optional i32 errorCode
+
+  // Error message
+  5: optional string errorMessage
+}
+
+
+// CancelOperation()
+//
+// Cancels processing on the specified operation handle and
+// frees any resources which were allocated.
+struct TCancelOperationReq {
+  // Operation to cancel
+  1: required TOperationHandle operationHandle
+}
+
+struct TCancelOperationResp {
+  1: required TStatus status
+}
+
+
+// CloseOperation()
+//
+// Given an operation in the FINISHED, CANCELED,
+// or ERROR states, CloseOperation() will free
+// all of the resources which were allocated on
+// the server to service the operation.
+struct TCloseOperationReq {
+  1: required TOperationHandle operationHandle
+}
+
+struct TCloseOperationResp {
+  1: required TStatus status
+}
+
+
+// GetResultSetMetadata()
+//
+// Retrieves schema information for the specified operation
+struct TGetResultSetMetadataReq {
+  // Operation for which to fetch result set schema information
+  1: required TOperationHandle operationHandle
+}
+
+struct TGetResultSetMetadataResp {
+  1: required TStatus status
+  2: optional TTableSchema schema
+}
+
+
+enum TFetchOrientation {
+  // Get the next rowset. The fetch offset is ignored.
+  FETCH_NEXT,
+
+  // Get the previous rowset. The fetch offset is ignored.
+  // NOT SUPPORTED
+  FETCH_PRIOR,
+
+  // Return the rowset at the given fetch offset relative
+  // to the curren rowset.
+  // NOT SUPPORTED
+  FETCH_RELATIVE,
+
+  // Return the rowset at the specified fetch offset.
+  // NOT SUPPORTED
+  FETCH_ABSOLUTE,
+
+  // Get the first rowset in the result set.
+  FETCH_FIRST,
+
+  // Get the last rowset in the result set.
+  // NOT SUPPORTED
+  FETCH_LAST
+}
+
+// FetchResults()
+//
+// Fetch rows from the server corresponding to
+// a particular OperationHandle.
+struct TFetchResultsReq {
+  // Operation from which to fetch results.
+  1: required TOperationHandle operationHandle
+
+  // The fetch orientation. For V1 this must be either
+  // FETCH_NEXT or FETCH_FIRST. Defaults to FETCH_NEXT.
+  2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT
+
+  // Max number of rows that should be returned in
+  // the rowset.
+  3: required i64 maxRows
+}
+
+struct TFetchResultsResp {
+  1: required TStatus status
+
+  // TRUE if there are more rows left to fetch from the server.
+  2: optional bool hasMoreRows
+
+  // The rowset. This is optional so that we have the
+  // option in the future of adding alternate formats for
+  // representing result set data, e.g. delimited strings,
+  // binary encoded, etc.
+  3: optional TRowSet results
+}
+
+// GetDelegationToken()
+// Retrieve delegation token for the current user
+struct  TGetDelegationTokenReq {
+  // session handle
+  1: required TSessionHandle sessionHandle
+
+  // userid for the proxy user
+  2: required string owner
+
+  // designated renewer userid
+  3: required string renewer
+}
+
+struct TGetDelegationTokenResp {
+  // status of the request
+  1: required TStatus status
+
+  // delegation token string
+  2: optional string delegationToken
+}
+
+// CancelDelegationToken()
+// Cancel the given delegation token
+struct TCancelDelegationTokenReq {
+  // session handle
+  1: required TSessionHandle sessionHandle
+
+  // delegation token to cancel
+  2: required string delegationToken
+}
+
+struct TCancelDelegationTokenResp {
+  // status of the request
+  1: required TStatus status
+}
+
+// RenewDelegationToken()
+// Renew the given delegation token
+struct TRenewDelegationTokenReq {
+  // session handle
+  1: required TSessionHandle sessionHandle
+
+  // delegation token to renew
+  2: required string delegationToken
+}
+
+struct TRenewDelegationTokenResp {
+  // status of the request
+  1: required TStatus status
+}
+
+// GetLog()
+// Not present in Hive 0.13, re-added for backwards compatibility.
+//
+// Fetch operation log from the server corresponding to
+// a particular OperationHandle.
+struct TGetLogReq {
+  // Operation whose log is requested
+  1: required TOperationHandle operationHandle
+}
+
+struct TGetLogResp {
+  1: required TStatus status
+  2: required string log
+}
+
+service TCLIService {
+
+  TOpenSessionResp OpenSession(1:TOpenSessionReq req);
+
+  TCloseSessionResp CloseSession(1:TCloseSessionReq req);
+
+  TGetInfoResp GetInfo(1:TGetInfoReq req);
+
+  TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req);
+
+  TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req);
+
+  TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req);
+
+  TGetSchemasResp GetSchemas(1:TGetSchemasReq req);
+
+  TGetTablesResp GetTables(1:TGetTablesReq req);
+
+  TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req);
+
+  TGetColumnsResp GetColumns(1:TGetColumnsReq req);
+
+  TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req);
+
+  TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req);
+
+  TCancelOperationResp CancelOperation(1:TCancelOperationReq req);
+
+  TCloseOperationResp CloseOperation(1:TCloseOperationReq req);
+
+  TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req);
+
+  TFetchResultsResp FetchResults(1:TFetchResultsReq req);
+
+  TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req);
+
+  TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req);
+
+  TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req);
+
+  // Not present in Hive 0.13, re-added for backwards compatibility.
+  TGetLogResp GetLog(1:TGetLogReq req);
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/Types.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/Types.thrift
new file mode 100644
index 0000000..4238f9c
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/Types.thrift
@@ -0,0 +1,218 @@
+// Copyright 2012 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.
+
+namespace cpp impala
+namespace java com.cloudera.impala.thrift
+
+typedef i64 TTimestamp
+typedef i32 TPlanNodeId
+typedef i32 TTupleId
+typedef i32 TSlotId
+typedef i32 TTableId
+
+// TODO: Consider moving unrelated enums to better locations.
+
+enum TPrimitiveType {
+  INVALID_TYPE,
+  NULL_TYPE,
+  BOOLEAN,
+  TINYINT,
+  SMALLINT,
+  INT,
+  BIGINT,
+  FLOAT,
+  DOUBLE,
+  DATE,
+  DATETIME,
+  TIMESTAMP,
+  STRING,
+  // Unsupported types
+  BINARY,
+  DECIMAL,
+  // CHAR(n). Currently only supported in UDAs
+  CHAR,
+  VARCHAR
+}
+
+enum TTypeNodeType {
+  SCALAR,
+  ARRAY,
+  MAP,
+  STRUCT
+}
+
+struct TScalarType {
+  1: required TPrimitiveType type
+
+  // Only set if type == CHAR or type == VARCHAR
+  2: optional i32 len
+
+  // Only set for DECIMAL
+  3: optional i32 precision
+  4: optional i32 scale
+}
+
+// Represents a field in a STRUCT type.
+// TODO: Model column stats for struct fields.
+struct TStructField {
+  1: required string name
+  2: optional string comment
+}
+
+struct TTypeNode {
+  1: required TTypeNodeType type
+
+  // only set for scalar types
+  2: optional TScalarType scalar_type
+
+  // only used for structs; has struct_fields.size() corresponding child types
+  3: optional list<TStructField> struct_fields
+}
+
+// A flattened representation of a tree of column types obtained by depth-first
+// traversal. Complex types such as map, array and struct have child types corresponding
+// to the map key/value, array item type, and struct fields, respectively.
+// For scalar types the list contains only a single node.
+// Note: We cannot rename this to TType because it conflicts with Thrift's internal TType
+// and the generated Python thrift files will not work.
+struct TColumnType {
+  1: list<TTypeNode> types
+}
+
+enum TStmtType {
+  QUERY,
+  DDL, // Data definition, e.g. CREATE TABLE (includes read-only functions e.g. SHOW)
+  DML, // Data modification e.g. INSERT
+  EXPLAIN,
+  LOAD, // Statement type for LOAD commands
+  SET
+}
+
+// Level of verboseness for "explain" output.
+enum TExplainLevel {
+  MINIMAL,
+  STANDARD,
+  EXTENDED,
+  VERBOSE
+}
+
+enum TRuntimeFilterMode {
+  // No filters are computed in the FE or the BE.
+  OFF,
+
+  // Only broadcast filters are computed in the BE, and are only published to the local
+  // fragment.
+  LOCAL,
+
+  // All fiters are computed in the BE, and are published globally.
+  GLOBAL
+}
+
+// A TNetworkAddress is the standard host, port representation of a
+// network address. The hostname field must be resolvable to an IPv4
+// address.
+struct TNetworkAddress {
+  1: required string hostname
+  2: required i32 port
+}
+
+// Wire format for UniqueId
+struct TUniqueId {
+  1: required i64 hi
+  2: required i64 lo
+}
+
+enum TFunctionCategory {
+  SCALAR,
+  AGGREGATE,
+  ANALYTIC
+}
+
+enum TFunctionBinaryType {
+  // Impala builtin. We can either run this interpreted or via codegen
+  // depending on the query option.
+  BUILTIN,
+
+  // Java UDFs, loaded from *.jar
+  JAVA,
+
+  // Native-interface, precompiled UDFs loaded from *.so
+  NATIVE,
+
+  // Native-interface, precompiled to IR; loaded from *.ll
+  IR,
+}
+
+// Represents a fully qualified function name.
+struct TFunctionName {
+  // Name of the function's parent database. Not set if in global
+  // namespace (e.g. builtins)
+  1: optional string db_name
+
+  // Name of the function
+  2: required string function_name
+}
+
+struct TScalarFunction {
+  1: required string symbol;
+  2: optional string prepare_fn_symbol
+  3: optional string close_fn_symbol
+}
+
+struct TAggregateFunction {
+  1: required TColumnType intermediate_type
+  2: required string update_fn_symbol
+  3: required string init_fn_symbol
+  4: optional string serialize_fn_symbol
+  5: optional string merge_fn_symbol
+  6: optional string finalize_fn_symbol
+  8: optional string get_value_fn_symbol
+  9: optional string remove_fn_symbol
+
+  7: optional bool ignores_distinct
+}
+
+// Represents a function in the Catalog.
+struct TFunction {
+  // Fully qualified function name.
+  1: required TFunctionName name
+
+  // Type of the udf. e.g. hive, native, ir
+  2: required TFunctionBinaryType binary_type
+
+  // The types of the arguments to the function
+  3: required list<TColumnType> arg_types
+
+  // Return type for the function.
+  4: required TColumnType ret_type
+
+  // If true, this function takes var args.
+  5: required bool has_var_args
+
+  // Optional comment to attach to the function
+  6: optional string comment
+
+  7: optional string signature
+
+  // HDFS path for the function binary. This binary must exist at the time the
+  // function is created.
+  8: optional string hdfs_location
+
+  // One of these should be set.
+  9: optional TScalarFunction scalar_fn
+  10: optional TAggregateFunction aggregate_fn
+
+  // True for builtins or user-defined functions persisted by the catalog
+  11: required bool is_persistent
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/beeswax.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/beeswax.thrift
new file mode 100644
index 0000000..a0ca5a7
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/beeswax.thrift
@@ -0,0 +1,174 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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.
+ *
+ * Interface for interacting with Beeswax Server
+ */
+
+namespace java com.cloudera.beeswax.api
+namespace py beeswaxd
+namespace cpp beeswax
+
+include "hive_metastore.thrift"
+
+// A Query
+struct Query {
+  1: string query;
+  // A list of HQL commands to execute before the query.
+  // This is typically defining UDFs, setting settings, and loading resources.
+  3: list<string> configuration;
+
+  // User and groups to "act as" for purposes of Hadoop.
+  4: string hadoop_user;
+}
+
+typedef string LogContextId
+
+enum QueryState {
+  CREATED,
+  INITIALIZED,
+  COMPILED,
+  RUNNING,
+  FINISHED,
+  EXCEPTION
+}
+
+struct QueryHandle {
+  1: string id;
+  2: LogContextId log_context;
+}
+
+struct QueryExplanation {
+  1: string textual
+}
+
+struct Results {
+  // If set, data is valid.  Otherwise, results aren't ready yet.
+  1: bool ready,
+  // Columns for the results
+  2: list<string> columns,
+  // A set of results
+  3: list<string> data,
+  // The starting row of the results
+  4: i64 start_row,
+  // Whether there are more results to fetch
+  5: bool has_more
+}
+
+/**
+ * Metadata information about the results.
+ * Applicable only for SELECT.
+ */
+struct ResultsMetadata {
+  /** The schema of the results */
+  1: hive_metastore.Schema schema,
+  /** The directory containing the results. Not applicable for partition table. */
+  2: string table_dir,
+  /** If the results are straight from an existing table, the table name. */
+  3: string in_tablename,
+  /** Field delimiter */
+  4: string delim,
+}
+
+exception BeeswaxException {
+  1: string message,
+  // Use get_log(log_context) to retrieve any log related to this exception
+  2: LogContextId log_context,
+  // (Optional) The QueryHandle that caused this exception
+  3: QueryHandle handle,
+  4: optional i32 errorCode = 0,
+  5: optional string SQLState = "     "
+}
+
+exception QueryNotFoundException {
+} 
+
+/** Represents a Hadoop-style configuration variable. */
+struct ConfigVariable {
+  1: string key,
+  2: string value,
+  3: string description
+}
+
+service BeeswaxService {
+  /**
+   * Submit a query and return a handle (QueryHandle). The query runs asynchronously.
+   */
+  QueryHandle query(1:Query query) throws(1:BeeswaxException error),
+
+  /**
+   * run a query synchronously and return a handle (QueryHandle).
+   */
+  QueryHandle executeAndWait(1:Query query, 2:LogContextId clientCtx) 
+                        throws(1:BeeswaxException error),
+
+  /**
+   * Get the query plan for a query.
+   */
+  QueryExplanation explain(1:Query query)
+                        throws(1:BeeswaxException error),
+
+  /**
+   * Get the results of a query. This is non-blocking. Caller should check
+   * Results.ready to determine if the results are in yet. The call requests
+   * the batch size of fetch.
+   */
+  Results fetch(1:QueryHandle query_id, 2:bool start_over, 3:i32 fetch_size=-1) 
+              throws(1:QueryNotFoundException error, 2:BeeswaxException error2),
+
+  /**
+   * Get the state of the query
+   */
+  QueryState get_state(1:QueryHandle handle) throws(1:QueryNotFoundException error),
+
+  /**
+   * Get the result metadata
+   */
+  ResultsMetadata get_results_metadata(1:QueryHandle handle)
+                                    throws(1:QueryNotFoundException error),
+
+  /**
+   * Used to test connection to server.  A "noop" command.
+   */
+  string echo(1:string s)
+
+  /**
+   * Returns a string representation of the configuration object being used.
+   * Handy for debugging.
+   */
+  string dump_config()
+
+  /**
+   * Get the log messages related to the given context.
+   */
+  string get_log(1:LogContextId context) throws(1:QueryNotFoundException error)
+
+  /*
+   * Returns "default" configuration.
+   */
+  list<ConfigVariable> get_default_configuration(1:bool include_hadoop)
+
+  /*
+   * closes the query with given handle
+   */
+  void close(1:QueryHandle handle) throws(1:QueryNotFoundException error, 
+                            2:BeeswaxException error2)
+
+  /*
+   * clean the log context for given id 
+   */
+  void clean(1:LogContextId log_context)
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/fb303.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/fb303.thrift
new file mode 100644
index 0000000..66c8315
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/fb303.thrift
@@ -0,0 +1,112 @@
+/*
+ * 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.
+ */
+
+/**
+ * fb303.thrift
+ */
+
+namespace java com.facebook.fb303
+namespace cpp facebook.fb303
+namespace perl Facebook.FB303
+
+/**
+ * Common status reporting mechanism across all services
+ */
+enum fb_status {
+  DEAD = 0,
+  STARTING = 1,
+  ALIVE = 2,
+  STOPPING = 3,
+  STOPPED = 4,
+  WARNING = 5,
+}
+
+/**
+ * Standard base service
+ */
+service FacebookService {
+
+  /**
+   * Returns a descriptive name of the service
+   */
+  string getName(),
+
+  /**
+   * Returns the version of the service
+   */
+  string getVersion(),
+
+  /**
+   * Gets the status of this service
+   */
+  fb_status getStatus(),
+
+  /**
+   * User friendly description of status, such as why the service is in
+   * the dead or warning state, or what is being started or stopped.
+   */
+  string getStatusDetails(),
+
+  /**
+   * Gets the counters for this service
+   */
+  map<string, i64> getCounters(),
+
+  /**
+   * Gets the value of a single counter
+   */
+  i64 getCounter(1: string key),
+
+  /**
+   * Sets an option
+   */
+  void setOption(1: string key, 2: string value),
+
+  /**
+   * Gets an option
+   */
+  string getOption(1: string key),
+
+  /**
+   * Gets all options
+   */
+  map<string, string> getOptions(),
+
+  /**
+   * Returns a CPU profile over the given time interval (client and server
+   * must agree on the profile format).
+   */
+  string getCpuProfile(1: i32 profileDurationInSec),
+
+  /**
+   * Returns the unix time that the server has been running since
+   */
+  i64 aliveSince(),
+
+  /**
+   * Tell the server to reload its configuration, reopen log files, etc
+   */
+  oneway void reinitialize(),
+
+  /**
+   * Suggest a shutdown to the server
+   */
+  oneway void shutdown(),
+
+}
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/generate_error_codes.py b/cpp/src/arrow/dbi/hiveserver2/thrift/generate_error_codes.py
new file mode 100644
index 0000000..3790057
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/generate_error_codes.py
@@ -0,0 +1,293 @@
+#!/usr/bin/env python
+# Copyright 2015 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.
+
+import sys
+import os
+
+
+# For readability purposes we define the error codes and messages at the top of the
+# file. New codes and messages must be added here. Old error messages MUST NEVER BE
+# DELETED, but can be renamed. The tuple layout for a new entry is: error code enum name,
+# numeric error code, format string of the message.
+#
+# TODO Add support for SQL Error Codes
+#      https://msdn.microsoft.com/en-us/library/ms714687%28v=vs.85%29.aspx
+error_codes = (
+  ("OK", 0, ""),
+
+  ("UNUSED", 1, "<UNUSED>"),
+
+  ("GENERAL", 2, "$0"),
+
+  ("CANCELLED", 3, "$0"),
+
+  ("ANALYSIS_ERROR", 4, "$0"),
+
+  ("NOT_IMPLEMENTED_ERROR", 5, "$0"),
+
+  ("RUNTIME_ERROR", 6, "$0"),
+
+  ("MEM_LIMIT_EXCEEDED", 7, "$0"),
+
+  ("INTERNAL_ERROR", 8, "$0"),
+
+  ("RECOVERABLE_ERROR", 9, "$0"),
+
+  ("PARQUET_MULTIPLE_BLOCKS", 10,
+   "Parquet files should not be split into multiple hdfs-blocks. file=$0"),
+
+  ("PARQUET_COLUMN_METADATA_INVALID", 11,
+   "Column metadata states there are $0 values, but read $1 values from column $2. "
+   "file=$3"),
+
+  ("PARQUET_HEADER_PAGE_SIZE_EXCEEDED", 12, "(unused)"),
+
+  ("PARQUET_HEADER_EOF", 13,
+    "ParquetScanner: reached EOF while deserializing data page header. file=$0"),
+
+  ("PARQUET_GROUP_ROW_COUNT_ERROR", 14,
+    "Metadata states that in group $0($1) there are $2 rows, but $3 rows were read."),
+
+  ("PARQUET_GROUP_ROW_COUNT_OVERFLOW", 15, "(unused)"),
+
+  ("PARQUET_MISSING_PRECISION", 16,
+   "File '$0' column '$1' does not have the decimal precision set."),
+
+  ("PARQUET_WRONG_PRECISION", 17,
+    "File '$0' column '$1' has a precision that does not match the table metadata "
+    " precision. File metadata precision: $2, table metadata precision: $3."),
+
+  ("PARQUET_BAD_CONVERTED_TYPE", 18,
+   "File '$0' column '$1' does not have converted type set to DECIMAL"),
+
+  ("PARQUET_INCOMPATIBLE_DECIMAL", 19,
+   "File '$0' column '$1' contains decimal data but the table metadata has type $2"),
+
+  ("SEQUENCE_SCANNER_PARSE_ERROR", 20,
+   "Problem parsing file $0 at $1$2"),
+
+  ("SNAPPY_DECOMPRESS_INVALID_BLOCK_SIZE", 21,
+   "Decompressor: block size is too big.  Data is likely corrupt. Size: $0"),
+
+  ("SNAPPY_DECOMPRESS_INVALID_COMPRESSED_LENGTH", 22,
+   "Decompressor: invalid compressed length.  Data is likely corrupt."),
+
+  ("SNAPPY_DECOMPRESS_UNCOMPRESSED_LENGTH_FAILED", 23,
+   "Snappy: GetUncompressedLength failed"),
+
+  ("SNAPPY_DECOMPRESS_RAW_UNCOMPRESS_FAILED", 24,
+   "SnappyBlock: RawUncompress failed"),
+
+  ("SNAPPY_DECOMPRESS_DECOMPRESS_SIZE_INCORRECT", 25,
+   "Snappy: Decompressed size is not correct."),
+
+  ("HDFS_SCAN_NODE_UNKNOWN_DISK", 26, "Unknown disk id.  "
+   "This will negatively affect performance. "
+   "Check your hdfs settings to enable block location metadata."),
+
+  ("FRAGMENT_EXECUTOR", 27, "Reserved resource size ($0) is larger than "
+    "query mem limit ($1), and will be restricted to $1. Configure the reservation "
+    "size by setting RM_INITIAL_MEM."),
+
+  ("PARTITIONED_HASH_JOIN_MAX_PARTITION_DEPTH", 28,
+   "Cannot perform join at hash join node with id $0."
+   " The input data was partitioned the maximum number of $1 times."
+   " This could mean there is significant skew in the data or the memory limit is"
+   " set too low."),
+
+  ("PARTITIONED_AGG_MAX_PARTITION_DEPTH", 29,
+   "Cannot perform aggregation at hash aggregation node with id $0."
+   " The input data was partitioned the maximum number of $1 times."
+   " This could mean there is significant skew in the data or the memory limit is"
+   " set too low."),
+
+  ("MISSING_BUILTIN", 30, "Builtin '$0' with symbol '$1' does not exist. "
+   "Verify that all your impalads are the same version."),
+
+  ("RPC_GENERAL_ERROR", 31, "RPC Error: $0"),
+  ("RPC_TIMEOUT", 32, "RPC timed out"),
+
+  ("UDF_VERIFY_FAILED", 33,
+   "Failed to verify function $0 from LLVM module $1, see log for more details."),
+
+  ("PARQUET_CORRUPT_VALUE", 34, "File $0 corrupt. RLE level data bytes = $1"),
+
+  ("AVRO_DECIMAL_RESOLUTION_ERROR", 35, "Column '$0' has conflicting Avro decimal types. "
+   "Table schema $1: $2, file schema $1: $3"),
+
+  ("AVRO_DECIMAL_METADATA_MISMATCH", 36, "Column '$0' has conflicting Avro decimal types. "
+   "Declared $1: $2, $1 in table's Avro schema: $3"),
+
+  ("AVRO_SCHEMA_RESOLUTION_ERROR", 37, "Unresolvable types for column '$0': "
+   "table type: $1, file type: $2"),
+
+  ("AVRO_SCHEMA_METADATA_MISMATCH", 38, "Unresolvable types for column '$0': "
+   "declared column type: $1, table's Avro schema type: $2"),
+
+  ("AVRO_UNSUPPORTED_DEFAULT_VALUE", 39, "Field $0 is missing from file and default "
+   "values of type $1 are not yet supported."),
+
+  ("AVRO_MISSING_FIELD", 40, "Inconsistent table metadata. Mismatch between column "
+   "definition and Avro schema: cannot read field $0 because there are only $1 fields."),
+
+  ("AVRO_MISSING_DEFAULT", 41,
+   "Field $0 is missing from file and does not have a default value."),
+
+  ("AVRO_NULLABILITY_MISMATCH", 42,
+   "Field $0 is nullable in the file schema but not the table schema."),
+
+  ("AVRO_NOT_A_RECORD", 43,
+   "Inconsistent table metadata. Field $0 is not a record in the Avro schema."),
+
+  ("PARQUET_DEF_LEVEL_ERROR", 44, "Could not read definition level, even though metadata"
+   " states there are $0 values remaining in data page. file=$1"),
+
+  ("PARQUET_NUM_COL_VALS_ERROR", 45, "Mismatched number of values in column index $0 "
+   "($1 vs. $2). file=$3"),
+
+  ("PARQUET_DICT_DECODE_FAILURE", 46, "Failed to decode dictionary-encoded value. "
+   "file=$0"),
+
+  ("SSL_PASSWORD_CMD_FAILED", 47,
+   "SSL private-key password command ('$0') failed with error: $1"),
+
+  ("SSL_CERTIFICATE_PATH_BLANK", 48, "The SSL certificate path is blank"),
+  ("SSL_PRIVATE_KEY_PATH_BLANK", 49, "The SSL private key path is blank"),
+
+  ("SSL_CERTIFICATE_NOT_FOUND", 50, "The SSL certificate file does not exist at path $0"),
+  ("SSL_PRIVATE_KEY_NOT_FOUND", 51, "The SSL private key file does not exist at path $0"),
+
+  ("SSL_SOCKET_CREATION_FAILED", 52, "SSL socket creation failed: $0"),
+
+  ("MEM_ALLOC_FAILED", 53, "Memory allocation of $0 bytes failed"),
+
+  ("PARQUET_REP_LEVEL_ERROR", 54, "Could not read repetition level, even though metadata"
+   " states there are $0 values remaining in data page. file=$1"),
+
+  ("PARQUET_UNRECOGNIZED_SCHEMA", 55, "File '$0' has an incompatible Parquet schema for "
+   "column '$1'. Column type: $2, Parquet schema:\\n$3"),
+
+  ("COLLECTION_ALLOC_FAILED", 56, "Failed to allocate buffer for collection '$0'."),
+
+  ("TMP_DEVICE_BLACKLISTED", 57,
+    "Temporary device for directory $0 is blacklisted from a previous error and cannot "
+    "be used."),
+
+  ("TMP_FILE_BLACKLISTED", 58,
+    "Temporary file $0 is blacklisted from a previous error and cannot be expanded."),
+
+  ("RPC_CLIENT_CONNECT_FAILURE", 59,
+    "RPC client failed to connect: $0"),
+
+  ("STALE_METADATA_FILE_TOO_SHORT", 60, "Metadata for file '$0' appears stale. "
+   "Try running \\\"refresh $1\\\" to reload the file metadata."),
+
+  ("PARQUET_BAD_VERSION_NUMBER", 61, "File '$0' has an invalid version number: $1\\n"
+   "This could be due to stale metadata. Try running \\\"refresh $2\\\"."),
+
+  ("SCANNER_INCOMPLETE_READ", 62, "Tried to read $0 bytes but could only read $1 bytes. "
+   "This may indicate data file corruption. (file $2, byte offset: $3)"),
+
+  ("SCANNER_INVALID_READ", 63, "Invalid read of $0 bytes. This may indicate data file "
+   "corruption. (file $1, byte offset: $2)"),
+
+  ("AVRO_BAD_VERSION_HEADER", 64, "File '$0' has an invalid version header: $1\\n"
+   "Make sure the file is an Avro data file."),
+
+  ("UDF_MEM_LIMIT_EXCEEDED", 65, "$0's allocations exceeded memory limits."),
+
+  ("BTS_BLOCK_OVERFLOW", 66, "Cannot process row that is bigger than the IO size "
+   "(row_size=$0, null_indicators_size=$1). To run this query, increase the IO size "
+   "(--read_size option)."),
+
+  ("COMPRESSED_FILE_MULTIPLE_BLOCKS", 67,
+   "For better performance, snappy-, gzip-, and bzip-compressed files "
+   "should not be split into multiple HDFS blocks. file=$0 offset $1"),
+
+  ("COMPRESSED_FILE_BLOCK_CORRUPTED", 68,
+   "$0 Data error, likely data corrupted in this block."),
+
+  ("COMPRESSED_FILE_DECOMPRESSOR_ERROR", 69, "$0 Decompressor error at $1, code=$2"),
+
+  ("COMPRESSED_FILE_DECOMPRESSOR_NO_PROGRESS", 70,
+   "Decompression failed to make progress, but end of input is not reached. "
+   "File appears corrupted. file=$0"),
+
+  ("COMPRESSED_FILE_TRUNCATED", 71,
+   "Unexpected end of compressed file. File may be truncated. file=$0")
+)
+
+# Verifies the uniqueness of the error constants and numeric error codes.
+# Numeric codes must start from 0, be in order and have no gaps
+def check_duplicates(codes):
+  constants = {}
+  next_num_code = 0
+  for row in codes:
+    if row[0] in constants:
+      print("Constant %s already used, please check definition of '%s'!" % \
+            (row[0], constants[row[0]]))
+      exit(1)
+    if row[1] != next_num_code:
+      print("Numeric error codes must start from 0, be in order, and not have any gaps: "
+            "got %d, expected %d" % (row[1], next_num_code))
+      exit(1)
+    next_num_code += 1
+    constants[row[0]] = row[2]
+
+preamble = """
+// Copyright 2015 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 AUTO GENERATED BY generated_error_codes.py DO NOT MODIFY
+// IT BY HAND.
+//
+
+namespace cpp impala
+namespace java com.cloudera.impala.thrift
+
+"""
+# The script will always generate the file, CMake will take care of running it only if
+# necessary.
+target_file = os.path.join(sys.argv[1], "ErrorCodes.thrift")
+
+# Check uniqueness of error constants and numeric codes
+check_duplicates(error_codes)
+
+fid = open(target_file, "w+")
+try:
+  fid.write(preamble)
+  fid.write("""\nenum TErrorCode {\n""")
+  fid.write(",\n".join(map(lambda x: "  %s = %d" % (x[0], x[1]), error_codes)))
+  fid.write("\n}")
+  fid.write("\n")
+  fid.write("const list<string> TErrorMessage = [\n")
+  fid.write(",\n".join(map(lambda x: "  // %s\n  \"%s\"" %(x[0], x[2]), error_codes)))
+  fid.write("\n]")
+finally:
+  fid.close()
+
+print("%s created." % target_file)
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift/hive_metastore.thrift b/cpp/src/arrow/dbi/hiveserver2/thrift/hive_metastore.thrift
new file mode 100644
index 0000000..f7c2693
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift/hive_metastore.thrift
@@ -0,0 +1,1214 @@
+#!/usr/local/bin/thrift -java
+
+/**
+ * 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.
+ */
+
+#
+# Thrift Service that the MetaStore is built on
+#
+
+include "fb303.thrift"
+
+namespace java org.apache.hadoop.hive.metastore.api
+namespace php metastore
+namespace cpp Apache.Hadoop.Hive
+
+const string DDL_TIME = "transient_lastDdlTime"
+
+struct Version {
+  1: string version,
+  2: string comments
+}
+
+struct FieldSchema {
+  1: string name, // name of the field
+  2: string type, // type of the field. primitive types defined above, specify list<TYPE_NAME>, map<TYPE_NAME, TYPE_NAME> for lists & maps
+  3: string comment
+}
+
+struct Type {
+  1: string          name,             // one of the types in PrimitiveTypes or CollectionTypes or User defined types
+  2: optional string type1,            // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE)
+  3: optional string type2,            // val type if the name is 'map' (MAP_TYPE)
+  4: optional list<FieldSchema> fields // if the name is one of the user defined types
+}
+
+enum HiveObjectType {
+  GLOBAL = 1,
+  DATABASE = 2,
+  TABLE = 3,
+  PARTITION = 4,
+  COLUMN = 5,
+}
+
+enum PrincipalType {
+  USER = 1,
+  ROLE = 2,
+  GROUP = 3,
+}
+
+const string HIVE_FILTER_FIELD_OWNER = "hive_filter_field_owner__"
+const string HIVE_FILTER_FIELD_PARAMS = "hive_filter_field_params__"
+const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"
+
+enum PartitionEventType {
+  LOAD_DONE = 1,
+}
+
+// Enums for transaction and lock management 
+enum TxnState {
+    COMMITTED = 1,
+    ABORTED = 2,
+    OPEN = 3,
+}
+
+enum LockLevel {
+    DB = 1,
+    TABLE = 2,
+    PARTITION = 3,
+}
+
+enum LockState {
+    ACQUIRED = 1,       // requester has the lock
+    WAITING = 2,        // requester is waiting for the lock and should call checklock at a later point to see if the lock has been obtained.
+    ABORT = 3,          // the lock has been aborted, most likely due to timeout
+    NOT_ACQUIRED = 4,   // returned only with lockNoWait, indicates the lock was not available and was not acquired
+}
+
+enum LockType {
+    SHARED_READ = 1,
+    SHARED_WRITE = 2,
+    EXCLUSIVE = 3,
+}
+
+enum CompactionType {
+    MINOR = 1,
+    MAJOR = 2,
+}
+
+enum GrantRevokeType {
+    GRANT = 1,
+    REVOKE = 2,
+}
+
+struct HiveObjectRef{
+  1: HiveObjectType objectType,
+  2: string dbName,
+  3: string objectName,
+  4: list<string> partValues,
+  5: string columnName,
+}
+
+struct PrivilegeGrantInfo {
+  1: string privilege,
+  2: i32 createTime,
+  3: string grantor,
+  4: PrincipalType grantorType,
+  5: bool grantOption,
+}
+
+struct HiveObjectPrivilege {
+  1: HiveObjectRef  hiveObject,
+  2: string principalName,
+  3: PrincipalType principalType,
+  4: PrivilegeGrantInfo grantInfo,
+}
+
+struct PrivilegeBag {
+  1: list<HiveObjectPrivilege> privileges,
+}
+
+struct PrincipalPrivilegeSet {
+  1: map<string, list<PrivilegeGrantInfo>> userPrivileges, // user name -> privilege grant info
+  2: map<string, list<PrivilegeGrantInfo>> groupPrivileges, // group name -> privilege grant info
+  3: map<string, list<PrivilegeGrantInfo>> rolePrivileges, //role name -> privilege grant info
+}
+
+struct GrantRevokePrivilegeRequest {
+  1: GrantRevokeType requestType;
+  2: PrivilegeBag privileges;
+  3: optional bool revokeGrantOption;  // Only for revoke request
+}
+
+struct GrantRevokePrivilegeResponse {
+  1: optional bool success;
+}
+
+struct Role {
+  1: string roleName,
+  2: i32 createTime,
+  3: string ownerName,
+}
+
+// Representation of a grant for a principal to a role
+struct RolePrincipalGrant {
+  1: string roleName,
+  2: string principalName,
+  3: PrincipalType principalType,
+  4: bool grantOption,
+  5: i32 grantTime,
+  6: string grantorName,
+  7: PrincipalType grantorPrincipalType
+}
+
+struct GetRoleGrantsForPrincipalRequest {
+  1: required string principal_name,
+  2: required PrincipalType principal_type
+}
+
+struct GetRoleGrantsForPrincipalResponse {
+  1: required list<RolePrincipalGrant> principalGrants;
+}
+
+struct GetPrincipalsInRoleRequest {
+  1: required string roleName;
+}
+
+struct GetPrincipalsInRoleResponse {
+  1: required list<RolePrincipalGrant> principalGrants;
+}
+
+struct GrantRevokeRoleRequest {
+  1: GrantRevokeType requestType;
+  2: string roleName;
+  3: string principalName;
+  4: PrincipalType principalType;
+  5: optional string grantor;            // Needed for grant
+  6: optional PrincipalType grantorType; // Needed for grant
+  7: optional bool grantOption;
+}
+
+struct GrantRevokeRoleResponse {
+  1: optional bool success;
+}
+
+// namespace for tables
+struct Database {
+  1: string name,
+  2: string description,
+  3: string locationUri,
+  4: map<string, string> parameters, // properties associated with the database
+  5: optional PrincipalPrivilegeSet privileges,
+  6: optional string ownerName,
+  7: optional PrincipalType ownerType
+}
+
+// This object holds the information needed by SerDes
+struct SerDeInfo {
+  1: string name,                   // name of the serde, table name by default
+  2: string serializationLib,       // usually the class that implements the extractor & loader
+  3: map<string, string> parameters // initialization parameters
+}
+
+// sort order of a column (column name along with asc(1)/desc(0))
+struct Order {
+  1: string col,  // sort column name
+  2: i32    order // asc(1) or desc(0)
+}
+
+// this object holds all the information about skewed table
+struct SkewedInfo {
+  1: list<string> skewedColNames, // skewed column names
+  2: list<list<string>> skewedColValues, //skewed values
+  3: map<list<string>, string> skewedColValueLocationMaps, //skewed value to location mappings
+}
+
+// this object holds all the information about physical storage of the data belonging to a table
+struct StorageDescriptor {
+  1: list<FieldSchema> cols,  // required (refer to types defined above)
+  2: string location,         // defaults to <warehouse loc>/<db loc>/tablename
+  3: string inputFormat,      // SequenceFileInputFormat (binary) or TextInputFormat`  or custom format
+  4: string outputFormat,     // SequenceFileOutputFormat (binary) or IgnoreKeyTextOutputFormat or custom format
+  5: bool   compressed,       // compressed or not
+  6: i32    numBuckets,       // this must be specified if there are any dimension columns
+  7: SerDeInfo    serdeInfo,  // serialization and deserialization information
+  8: list<string> bucketCols, // reducer grouping columns and clustering columns and bucketing columns`
+  9: list<Order>  sortCols,   // sort order of the data in each bucket
+  10: map<string, string> parameters, // any user supplied key value hash
+  11: optional SkewedInfo skewedInfo, // skewed information
+  12: optional bool   storedAsSubDirectories       // stored as subdirectories or not
+}
+
+// table information
+struct Table {
+  1: string tableName,                // name of the table
+  2: string dbName,                   // database name ('default')
+  3: string owner,                    // owner of this table
+  4: i32    createTime,               // creation time of the table
+  5: i32    lastAccessTime,           // last access time (usually this will be filled from HDFS and shouldn't be relied on)
+  6: i32    retention,                // retention time
+  7: StorageDescriptor sd,            // storage descriptor of the table
+  8: list<FieldSchema> partitionKeys, // partition keys of the table. only primitive types are supported
+  9: map<string, string> parameters,   // to store comments or any other user level parameters
+  10: string viewOriginalText,         // original view text, null for non-view
+  11: string viewExpandedText,         // expanded view text, null for non-view
+  12: string tableType,                 // table type enum, e.g. EXTERNAL_TABLE
+  13: optional PrincipalPrivilegeSet privileges,
+  14: optional bool temporary=false
+}
+
+struct Partition {
+  1: list<string> values // string value is converted to appropriate partition key type
+  2: string       dbName,
+  3: string       tableName,
+  4: i32          createTime,
+  5: i32          lastAccessTime,
+  6: StorageDescriptor   sd,
+  7: map<string, string> parameters,
+  8: optional PrincipalPrivilegeSet privileges
+}
+
+struct PartitionWithoutSD {
+  1: list<string> values // string value is converted to appropriate partition key type
+  2: i32          createTime,
+  3: i32          lastAccessTime,
+  4: string       relativePath,
+  5: map<string, string> parameters,
+  6: optional PrincipalPrivilegeSet privileges
+}
+
+struct PartitionSpecWithSharedSD {
+  1: list<PartitionWithoutSD> partitions,
+  2: StorageDescriptor sd,
+}
+
+struct PartitionListComposingSpec {
+  1: list<Partition> partitions
+}
+
+struct PartitionSpec {
+  1: string dbName,
+  2: string tableName,
+  3: string rootPath,
+  4: optional PartitionSpecWithSharedSD sharedSDPartitionSpec,
+  5: optional PartitionListComposingSpec partitionList
+}
+
+struct Index {
+  1: string       indexName, // unique with in the whole database namespace
+  2: string       indexHandlerClass, // reserved
+  3: string       dbName,
+  4: string       origTableName,
+  5: i32          createTime,
+  6: i32          lastAccessTime,
+  7: string       indexTableName,
+  8: StorageDescriptor   sd,
+  9: map<string, string> parameters,
+  10: bool         deferredRebuild
+}
+
+// column statistics
+struct BooleanColumnStatsData {
+1: required i64 numTrues,
+2: required i64 numFalses,
+3: required i64 numNulls
+}
+
+struct DoubleColumnStatsData {
+1: optional double lowValue,
+2: optional double highValue,
+3: required i64 numNulls,
+4: required i64 numDVs
+}
+
+struct LongColumnStatsData {
+1: optional i64 lowValue,
+2: optional i64 highValue,
+3: required i64 numNulls,
+4: required i64 numDVs
+}
+
+struct StringColumnStatsData {
+1: required i64 maxColLen,
+2: required double avgColLen,
+3: required i64 numNulls,
+4: required i64 numDVs
+}
+
+struct BinaryColumnStatsData {
+1: required i64 maxColLen,
+2: required double avgColLen,
+3: required i64 numNulls
+}
+
+
+struct Decimal {
+1: required binary unscaled,
+3: required i16 scale
+}
+
+struct DecimalColumnStatsData {
+1: optional Decimal lowValue,
+2: optional Decimal highValue,
+3: required i64 numNulls,
+4: required i64 numDVs
+}
+
+union ColumnStatisticsData {
+1: BooleanColumnStatsData booleanStats,
+2: LongColumnStatsData longStats,
+3: DoubleColumnStatsData doubleStats,
+4: StringColumnStatsData stringStats,
+5: BinaryColumnStatsData binaryStats,
+6: DecimalColumnStatsData decimalStats
+}
+
+struct ColumnStatisticsObj {
+1: required string colName,
+2: required string colType,
+3: required ColumnStatisticsData statsData
+}
+
+struct ColumnStatisticsDesc {
+1: required bool isTblLevel,
+2: required string dbName,
+3: required string tableName,
+4: optional string partName,
+5: optional i64 lastAnalyzed
+}
+
+struct ColumnStatistics {
+1: required ColumnStatisticsDesc statsDesc,
+2: required list<ColumnStatisticsObj> statsObj;
+}
+
+struct AggrStats {
+1: required list<ColumnStatisticsObj> colStats,
+2: required i64 partsFound // number of partitions for which stats were found
+}
+
+struct SetPartitionsStatsRequest {
+1: required list<ColumnStatistics> colStats
+}
+
+// schema of the table/query results etc.
+struct Schema {
+ // column names, types, comments
+ 1: list<FieldSchema> fieldSchemas,  // delimiters etc
+ 2: map<string, string> properties
+}
+
+// Key-value store to be used with selected
+// Metastore APIs (create, alter methods).
+// The client can pass environment properties / configs that can be
+// accessed in hooks.
+struct EnvironmentContext {
+  1: map<string, string> properties
+}
+
+// Return type for get_partitions_by_expr
+struct PartitionsByExprResult {
+  1: required list<Partition> partitions,
+  // Whether the results has any (currently, all) partitions which may or may not match
+  2: required bool hasUnknownPartitions
+}
+
+struct PartitionsByExprRequest {
+  1: required string dbName,
+  2: required string tblName,
+  3: required binary expr,
+  4: optional string defaultPartitionName,
+  5: optional i16 maxParts=-1
+}
+
+struct TableStatsResult {
+  1: required list<ColumnStatisticsObj> tableStats
+}
+
+struct PartitionsStatsResult {
+  1: required map<string, list<ColumnStatisticsObj>> partStats
+}
+
+struct TableStatsRequest {
+ 1: required string dbName,
+ 2: required string tblName,
+ 3: required list<string> colNames
+}
+
+struct PartitionsStatsRequest {
+ 1: required string dbName,
+ 2: required string tblName,
+ 3: required list<string> colNames,
+ 4: required list<string> partNames
+}
+
+// Return type for add_partitions_req
+struct AddPartitionsResult {
+  1: optional list<Partition> partitions,
+}
+
+// Request type for add_partitions_req
+struct AddPartitionsRequest {
+  1: required string dbName,
+  2: required string tblName,
+  3: required list<Partition> parts,
+  4: required bool ifNotExists,
+  5: optional bool needResult=true
+}
+
+// Return type for drop_partitions_req
+struct DropPartitionsResult {
+  1: optional list<Partition> partitions,
+}
+
+struct DropPartitionsExpr {
+  1: required binary expr;
+  2: optional i32 partArchiveLevel;
+}
+
+union RequestPartsSpec {
+  1: list<string> names;
+  2: list<DropPartitionsExpr> exprs;
+}
+
+// Request type for drop_partitions_req
+// TODO: we might want to add "bestEffort" flag; where a subset can fail
+struct DropPartitionsRequest {
+  1: required string dbName,
+  2: required string tblName,
+  3: required RequestPartsSpec parts,
+  4: optional bool deleteData,
+  5: optional bool ifExists=true, // currently verified on client
+  6: optional bool ignoreProtection,
+  7: optional EnvironmentContext environmentContext,
+  8: optional bool needResult=true
+}
+
+enum FunctionType {
+  JAVA = 1,
+}
+
+enum ResourceType {
+  JAR     = 1,
+  FILE    = 2,
+  ARCHIVE = 3,
+}
+
+struct ResourceUri {
+  1: ResourceType resourceType,
+  2: string       uri,
+}
+
+// User-defined function
+struct Function {
+  1: string           functionName,
+  2: string           dbName,
+  3: string           className,
+  4: string           ownerName,
+  5: PrincipalType    ownerType,
+  6: i32              createTime,
+  7: FunctionType     functionType,
+  8: list<ResourceUri> resourceUris,
+}
+
+// Structs for transaction and locks
+struct TxnInfo {
+    1: required i64 id,
+    2: required TxnState state,
+    3: required string user,        // used in 'show transactions' to help admins find who has open transactions
+    4: required string hostname,    // used in 'show transactions' to help admins find who has open transactions
+}
+
+struct GetOpenTxnsInfoResponse {
+    1: required i64 txn_high_water_mark,
+    2: required list<TxnInfo> open_txns,
+}
+
+struct GetOpenTxnsResponse {
+    1: required i64 txn_high_water_mark,
+    2: required set<i64> open_txns,
+}
+
+struct OpenTxnRequest {
+    1: required i32 num_txns,
+    2: required string user,
+    3: required string hostname,
+}
+
+struct OpenTxnsResponse {
+    1: required list<i64> txn_ids,
+}
+
+struct AbortTxnRequest {
+    1: required i64 txnid,
+}
+
+struct CommitTxnRequest {
+    1: required i64 txnid,
+}
+
+struct LockComponent {
+    1: required LockType type,
+    2: required LockLevel level,
+    3: required string dbname,
+    4: optional string tablename,
+    5: optional string partitionname,
+}
+
+struct LockRequest {
+    1: required list<LockComponent> component,
+    2: optional i64 txnid,
+    3: required string user,     // used in 'show locks' to help admins find who has open locks
+    4: required string hostname, // used in 'show locks' to help admins find who has open locks
+}
+
+struct LockResponse {
+    1: required i64 lockid,
+    2: required LockState state,
+}
+
+struct CheckLockRequest {
+    1: required i64 lockid,
+}
+
+struct UnlockRequest {
+    1: required i64 lockid,
+}
+
+struct ShowLocksRequest {
+}
+
+struct ShowLocksResponseElement {
+    1: required i64 lockid,
+    2: required string dbname,
+    3: optional string tablename,
+    4: optional string partname,
+    5: required LockState state,
+    6: required LockType type,
+    7: optional i64 txnid,
+    8: required i64 lastheartbeat,
+    9: optional i64 acquiredat,
+    10: required string user,
+    11: required string hostname,
+}
+
+struct ShowLocksResponse {
+    1: list<ShowLocksResponseElement> locks,
+}
+
+struct HeartbeatRequest {
+    1: optional i64 lockid,
+    2: optional i64 txnid
+}
+
+struct HeartbeatTxnRangeRequest {
+    1: required i64 min,
+    2: required i64 max
+}
+
+struct HeartbeatTxnRangeResponse {
+    1: required set<i64> aborted,
+    2: required set<i64> nosuch
+}
+
+struct CompactionRequest {
+    1: required string dbname,
+    2: required string tablename,
+    3: optional string partitionname,
+    4: required CompactionType type,
+    5: optional string runas,
+}
+
+struct ShowCompactRequest {
+}
+
+struct ShowCompactResponseElement {
+    1: required string dbname,
+    2: required string tablename,
+    3: optional string partitionname,
+    4: required CompactionType type,
+    5: required string state,
+    6: optional string workerid,
+    7: optional i64 start,
+    8: optional string runAs,
+}
+
+struct ShowCompactResponse {
+    1: required list<ShowCompactResponseElement> compacts,
+}
+
+struct NotificationEventRequest {
+    1: required i64 lastEvent,
+    2: optional i32 maxEvents,
+}
+
+struct NotificationEvent {
+    1: required i64 eventId,
+    2: required i32 eventTime,
+    3: required string eventType,
+    4: optional string dbName,
+    5: optional string tableName,
+    6: required string message,
+}
+
+struct NotificationEventResponse {
+    1: required list<NotificationEvent> events,
+}
+
+struct CurrentNotificationEventId {
+    1: required i64 eventId,
+}
+
+struct InsertEventRequestData {
+    1: required list<string> filesAdded
+}
+
+union FireEventRequestData {
+    1: InsertEventRequestData insertData
+}
+
+struct FireEventRequest {
+    1: required bool successful,
+    2: required FireEventRequestData data
+    // dbname, tablename, and partition vals are included as optional in the top level event rather than placed in each type of
+    // subevent as I assume they'll be used across most event types.
+    3: optional string dbName,
+    4: optional string tableName,
+    5: optional list<string> partitionVals,
+}
+
+struct FireEventResponse {
+    // NOP for now, this is just a place holder for future responses
+}
+    
+
+struct GetAllFunctionsResponse {
+  1: optional list<Function> functions
+}
+
+struct TableMeta {
+  1: required string dbName;
+  2: required string tableName;
+  3: required string tableType;
+  4: optional string comments;
+}
+
+exception MetaException {
+  1: string message
+}
+
+exception UnknownTableException {
+  1: string message
+}
+
+exception UnknownDBException {
+  1: string message
+}
+
+exception AlreadyExistsException {
+  1: string message
+}
+
+exception InvalidPartitionException {
+  1: string message
+}
+
+exception UnknownPartitionException {
+  1: string message
+}
+
+exception InvalidObjectException {
+  1: string message
+}
+
+exception NoSuchObjectException {
+  1: string message
+}
+
+exception IndexAlreadyExistsException {
+  1: string message
+}
+
+exception InvalidOperationException {
+  1: string message
+}
+
+exception ConfigValSecurityException {
+  1: string message
+}
+
+exception InvalidInputException {
+  1: string message
+}
+
+// Transaction and lock exceptions
+exception NoSuchTxnException {
+    1: string message
+}
+
+exception TxnAbortedException {
+    1: string message
+}
+
+exception TxnOpenException {
+    1: string message
+}
+
+exception NoSuchLockException {
+    1: string message
+}
+
+/**
+* This interface is live.
+*/
+service ThriftHiveMetastore extends fb303.FacebookService
+{
+  string getMetaConf(1:string key) throws(1:MetaException o1)
+  void setMetaConf(1:string key, 2:string value) throws(1:MetaException o1)
+
+  void create_database(1:Database database) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3)
+  Database get_database(1:string name) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  void drop_database(1:string name, 2:bool deleteData, 3:bool cascade) throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3)
+  list<string> get_databases(1:string pattern) throws(1:MetaException o1)
+  list<string> get_all_databases() throws(1:MetaException o1)
+  void alter_database(1:string dbname, 2:Database db) throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // returns the type with given name (make seperate calls for the dependent types if needed)
+  Type get_type(1:string name)  throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  bool create_type(1:Type type) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3)
+  bool drop_type(1:string type) throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  map<string, Type> get_type_all(1:string name)
+                                throws(1:MetaException o2)
+
+  // Gets a list of FieldSchemas describing the columns of a particular table
+  list<FieldSchema> get_fields(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3),
+  list<FieldSchema> get_fields_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3)
+
+  // Gets a list of FieldSchemas describing both the columns and the partition keys of a particular table
+  list<FieldSchema> get_schema(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3)
+  list<FieldSchema> get_schema_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3)
+
+  // create a Hive table. Following fields must be set
+  // tableName
+  // database        (only 'default' for now until Hive QL supports databases)
+  // owner           (not needed, but good to have for tracking purposes)
+  // sd.cols         (list of field schemas)
+  // sd.inputFormat  (SequenceFileInputFormat (binary like falcon tables or u_full) or TextInputFormat)
+  // sd.outputFormat (SequenceFileInputFormat (binary) or TextInputFormat)
+  // sd.serdeInfo.serializationLib (SerDe class name eg org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe
+  // * See notes on DDL_TIME
+  void create_table(1:Table tbl) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:NoSuchObjectException o4)
+  void create_table_with_environment_context(1:Table tbl,
+      2:EnvironmentContext environment_context)
+      throws (1:AlreadyExistsException o1,
+              2:InvalidObjectException o2, 3:MetaException o3,
+              4:NoSuchObjectException o4)
+  // drops the table and all the partitions associated with it if the table has partitions
+  // delete data (including partitions) if deleteData is set to true
+  void drop_table(1:string dbname, 2:string name, 3:bool deleteData)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o3)
+  void drop_table_with_environment_context(1:string dbname, 2:string name, 3:bool deleteData,
+      4:EnvironmentContext environment_context)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o3)
+  list<string> get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1)
+  list<TableMeta> get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list<string> tbl_types)
+                       throws (1: MetaException o1)
+  list<string> get_all_tables(1: string db_name) throws (1: MetaException o1)
+
+  Table get_table(1:string dbname, 2:string tbl_name)
+                       throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  list<Table> get_table_objects_by_name(1:string dbname, 2:list<string> tbl_names)
+				   throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
+
+  // Get a list of table names that match a filter.
+  // The filter operators are LIKE, <, <=, >, >=, =, <>
+  //
+  // In the filter statement, values interpreted as strings must be enclosed in quotes,
+  // while values interpreted as integers should not be.  Strings and integers are the only
+  // supported value types.
+  //
+  // The currently supported key names in the filter are:
+  // Constants.HIVE_FILTER_FIELD_OWNER, which filters on the tables' owner's name
+  //   and supports all filter operators
+  // Constants.HIVE_FILTER_FIELD_LAST_ACCESS, which filters on the last access times
+  //   and supports all filter operators except LIKE
+  // Constants.HIVE_FILTER_FIELD_PARAMS, which filters on the tables' parameter keys and values
+  //   and only supports the filter operators = and <>.
+  //   Append the parameter key name to HIVE_FILTER_FIELD_PARAMS in the filter statement.
+  //   For example, to filter on parameter keys called "retention", the key name in the filter
+  //   statement should be Constants.HIVE_FILTER_FIELD_PARAMS + "retention"
+  //   Also, = and <> only work for keys that exist
+  //   in the tables. E.g., if you are looking for tables where key1 <> value, it will only
+  //   look at tables that have a value for the parameter key1.
+  // Some example filter statements include:
+  // filter = Constants.HIVE_FILTER_FIELD_OWNER + " like \".*test.*\" and " +
+  //   Constants.HIVE_FILTER_FIELD_LAST_ACCESS + " = 0";
+  // filter = Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"30\" or " +
+  //   Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"90\""
+  // @param dbName
+  //          The name of the database from which you will retrieve the table names
+  // @param filterType
+  //          The type of filter
+  // @param filter
+  //          The filter string
+  // @param max_tables
+  //          The maximum number of tables returned
+  // @return  A list of table names that match the desired filter
+  list<string> get_table_names_by_filter(1:string dbname, 2:string filter, 3:i16 max_tables=-1)
+                       throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
+
+  // alter table applies to only future partitions not for existing partitions
+  // * See notes on DDL_TIME
+  void alter_table(1:string dbname, 2:string tbl_name, 3:Table new_tbl)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+  void alter_table_with_environment_context(1:string dbname, 2:string tbl_name,
+      3:Table new_tbl, 4:EnvironmentContext environment_context)
+      throws (1:InvalidOperationException o1, 2:MetaException o2)
+  // alter table not only applies to future partitions but also cascade to existing partitions
+  void alter_table_with_cascade(1:string dbname, 2:string tbl_name, 3:Table new_tbl, 4:bool cascade)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+  // the following applies to only tables that have partitions
+  // * See notes on DDL_TIME
+  Partition add_partition(1:Partition new_part)
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition add_partition_with_environment_context(1:Partition new_part,
+      2:EnvironmentContext environment_context)
+      throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2,
+      3:MetaException o3)
+  i32 add_partitions(1:list<Partition> new_parts)
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  i32 add_partitions_pspec(1:list<PartitionSpec> new_parts)
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition append_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals)
+                       throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  AddPartitionsResult add_partitions_req(1:AddPartitionsRequest request)
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition append_partition_with_environment_context(1:string db_name, 2:string tbl_name,
+      3:list<string> part_vals, 4:EnvironmentContext environment_context)
+                       throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition append_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name)
+                       throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition append_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name,
+      3:string part_name, 4:EnvironmentContext environment_context)
+                       throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  bool drop_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:bool deleteData)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  bool drop_partition_with_environment_context(1:string db_name, 2:string tbl_name,
+      3:list<string> part_vals, 4:bool deleteData, 5:EnvironmentContext environment_context)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  bool drop_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name, 4:bool deleteData)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  bool drop_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name,
+      3:string part_name, 4:bool deleteData, 5:EnvironmentContext environment_context)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  DropPartitionsResult drop_partitions_req(1: DropPartitionsRequest req)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
+  Partition get_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  Partition exchange_partition(1:map<string, string> partitionSpecs, 2:string source_db,
+      3:string source_table_name, 4:string dest_db, 5:string dest_table_name)
+      throws(1:MetaException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3,
+      4:InvalidInputException o4)
+
+  Partition get_partition_with_auth(1:string db_name, 2:string tbl_name, 3:list<string> part_vals,
+      4: string user_name, 5: list<string> group_names) throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  Partition get_partition_by_name(1:string db_name 2:string tbl_name, 3:string part_name)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // returns all the partitions for this table in reverse chronological order.
+  // If max parts is given then it will return only that many.
+  list<Partition> get_partitions(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  list<Partition> get_partitions_with_auth(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1,
+     4: string user_name, 5: list<string> group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
+  list<PartitionSpec> get_partitions_pspec(1:string db_name, 2:string tbl_name, 3:i32 max_parts=-1)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
+  list<string> get_partition_names(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1)
+                       throws(1:MetaException o2)
+
+  // get_partition*_ps methods allow filtering by a partial partition specification,
+  // as needed for dynamic partitions. The values that are not restricted should
+  // be empty strings. Nulls were considered (instead of "") but caused errors in
+  // generated Python code. The size of part_vals may be smaller than the
+  // number of partition columns - the unspecified values are considered the same
+  // as "".
+  list<Partition> get_partitions_ps(1:string db_name 2:string tbl_name
+  	3:list<string> part_vals, 4:i16 max_parts=-1)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  list<Partition> get_partitions_ps_with_auth(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:i16 max_parts=-1,
+     5: string user_name, 6: list<string> group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
+  list<string> get_partition_names_ps(1:string db_name,
+  	2:string tbl_name, 3:list<string> part_vals, 4:i16 max_parts=-1)
+  	                   throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // get the partitions matching the given partition filter
+  list<Partition> get_partitions_by_filter(1:string db_name 2:string tbl_name
+    3:string filter, 4:i16 max_parts=-1)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // List partitions as PartitionSpec instances.
+  list<PartitionSpec> get_part_specs_by_filter(1:string db_name 2:string tbl_name
+    3:string filter, 4:i32 max_parts=-1)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // get the partitions matching the given partition filter
+  // unlike get_partitions_by_filter, takes serialized hive expression, and with that can work
+  // with any filter (get_partitions_by_filter only works if the filter can be pushed down to JDOQL.
+  PartitionsByExprResult get_partitions_by_expr(1:PartitionsByExprRequest req)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // get partitions give a list of partition names
+  list<Partition> get_partitions_by_names(1:string db_name 2:string tbl_name 3:list<string> names)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  // changes the partition to the new partition object. partition is identified from the part values
+  // in the new_part
+  // * See notes on DDL_TIME
+  void alter_partition(1:string db_name, 2:string tbl_name, 3:Partition new_part)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+
+  // change a list of partitions. All partitions are altered atomically and all
+  // prehooks are fired together followed by all post hooks
+  void alter_partitions(1:string db_name, 2:string tbl_name, 3:list<Partition> new_parts)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+
+  void alter_partition_with_environment_context(1:string db_name,
+      2:string tbl_name, 3:Partition new_part,
+      4:EnvironmentContext environment_context)
+      throws (1:InvalidOperationException o1, 2:MetaException o2)
+
+  // rename the old partition to the new partition object by changing old part values to the part values
+  // in the new_part. old partition is identified from part_vals.
+  // partition keys in new_part should be the same as those in old partition.
+  void rename_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:Partition new_part)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+
+  // returns whether or not the partition name is valid based on the value of the config
+  // hive.metastore.partition.name.whitelist.pattern
+  bool partition_name_has_valid_characters(1:list<string> part_vals, 2:bool throw_exception)
+ 	throws(1: MetaException o1)
+
+  // gets the value of the configuration key in the metastore server. returns
+  // defaultValue if the key does not exist. if the configuration key does not
+  // begin with "hive", "mapred", or "hdfs", a ConfigValSecurityException is
+  // thrown.
+  string get_config_value(1:string name, 2:string defaultValue)
+                          throws(1:ConfigValSecurityException o1)
+
+  // converts a partition name into a partition values array
+  list<string> partition_name_to_vals(1: string part_name)
+                          throws(1: MetaException o1)
+  // converts a partition name into a partition specification (a mapping from
+  // the partition cols to the values)
+  map<string, string> partition_name_to_spec(1: string part_name)
+                          throws(1: MetaException o1)
+
+  void markPartitionForEvent(1:string db_name, 2:string tbl_name, 3:map<string,string> part_vals,
+                  4:PartitionEventType eventType) throws (1: MetaException o1, 2: NoSuchObjectException o2,
+                  3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5,
+                  6: InvalidPartitionException o6)
+  bool isPartitionMarkedForEvent(1:string db_name, 2:string tbl_name, 3:map<string,string> part_vals,
+                  4: PartitionEventType eventType) throws (1: MetaException o1, 2:NoSuchObjectException o2,
+                  3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5,
+                  6: InvalidPartitionException o6)
+
+  //index
+  Index add_index(1:Index new_index, 2: Table index_table)
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  void alter_index(1:string dbname, 2:string base_tbl_name, 3:string idx_name, 4:Index new_idx)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+  bool drop_index_by_name(1:string db_name, 2:string tbl_name, 3:string index_name, 4:bool deleteData)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  Index get_index_by_name(1:string db_name 2:string tbl_name, 3:string index_name)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+
+  list<Index> get_indexes(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  list<string> get_index_names(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1)
+                       throws(1:MetaException o2)
+
+  // column statistics interfaces
+
+  // update APIs persist the column statistics object(s) that are passed in. If statistics already
+  // exists for one or more columns, the existing statistics will be overwritten. The update APIs
+  // validate that the dbName, tableName, partName, colName[] passed in as part of the ColumnStatistics
+  // struct are valid, throws InvalidInputException/NoSuchObjectException if found to be invalid
+  bool update_table_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1,
+              2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4)
+  bool update_partition_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1,
+              2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4)
+
+  // get APIs return the column statistics corresponding to db_name, tbl_name, [part_name], col_name if
+  // such statistics exists. If the required statistics doesn't exist, get APIs throw NoSuchObjectException
+  // For instance, if get_table_column_statistics is called on a partitioned table for which only
+  // partition level column stats exist, get_table_column_statistics will throw NoSuchObjectException
+  ColumnStatistics get_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidInputException o3, 4:InvalidObjectException o4)
+  ColumnStatistics get_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name,
+               4:string col_name) throws (1:NoSuchObjectException o1, 2:MetaException o2,
+               3:InvalidInputException o3, 4:InvalidObjectException o4)
+  TableStatsResult get_table_statistics_req(1:TableStatsRequest request) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2)
+  PartitionsStatsResult get_partitions_statistics_req(1:PartitionsStatsRequest request) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2)
+  AggrStats get_aggr_stats_for(1:PartitionsStatsRequest request) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2)
+  bool set_aggr_stats_for(1:SetPartitionsStatsRequest request) throws
+              (1:NoSuchObjectException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4)
+
+
+  // delete APIs attempt to delete column statistics, if found, associated with a given db_name, tbl_name, [part_name]
+  // and col_name. If the delete API doesn't find the statistics record in the metastore, throws NoSuchObjectException
+  // Delete API validates the input and if the input is invalid throws InvalidInputException/InvalidObjectException.
+  bool delete_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name, 4:string col_name) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3,
+               4:InvalidInputException o4)
+  bool delete_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws
+              (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3,
+               4:InvalidInputException o4)
+
+  //
+  // user-defined functions
+  //
+
+  void create_function(1:Function func)
+      throws (1:AlreadyExistsException o1,
+              2:InvalidObjectException o2,
+              3:MetaException o3,
+              4:NoSuchObjectException o4)
+
+  void drop_function(1:string dbName, 2:string funcName)
+      throws (1:NoSuchObjectException o1, 2:MetaException o3)
+
+  void alter_function(1:string dbName, 2:string funcName, 3:Function newFunc)
+      throws (1:InvalidOperationException o1, 2:MetaException o2)
+
+  list<string> get_functions(1:string dbName, 2:string pattern)
+      throws (1:MetaException o1)
+  Function get_function(1:string dbName, 2:string funcName)
+      throws (1:MetaException o1, 2:NoSuchObjectException o2)
+
+  GetAllFunctionsResponse get_all_functions() throws (1:MetaException o1)
+
+  //authorization privileges
+
+  bool create_role(1:Role role) throws(1:MetaException o1)
+  bool drop_role(1:string role_name) throws(1:MetaException o1)
+  list<string> get_role_names() throws(1:MetaException o1)
+  // Deprecated, use grant_revoke_role()
+  bool grant_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type,
+    4:string grantor, 5:PrincipalType grantorType, 6:bool grant_option) throws(1:MetaException o1)
+  // Deprecated, use grant_revoke_role()
+  bool revoke_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type)
+                        throws(1:MetaException o1)
+  list<Role> list_roles(1:string principal_name, 2:PrincipalType principal_type) throws(1:MetaException o1)
+  GrantRevokeRoleResponse grant_revoke_role(1:GrantRevokeRoleRequest request) throws(1:MetaException o1)
+
+  // get all role-grants for users/roles that have been granted the given role
+  // Note that in the returned list of RolePrincipalGrants, the roleName is
+  // redundant as it would match the role_name argument of this function
+  GetPrincipalsInRoleResponse get_principals_in_role(1: GetPrincipalsInRoleRequest request) throws(1:MetaException o1)
+
+  // get grant information of all roles granted to the given principal
+  // Note that in the returned list of RolePrincipalGrants, the principal name,type is
+  // redundant as it would match the principal name,type arguments of this function
+  GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(1: GetRoleGrantsForPrincipalRequest request) throws(1:MetaException o1)
+
+  PrincipalPrivilegeSet get_privilege_set(1:HiveObjectRef hiveObject, 2:string user_name,
+    3: list<string> group_names) throws(1:MetaException o1)
+  list<HiveObjectPrivilege> list_privileges(1:string principal_name, 2:PrincipalType principal_type,
+    3: HiveObjectRef hiveObject) throws(1:MetaException o1)
+
+  // Deprecated, use grant_revoke_privileges()
+  bool grant_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1)
+  // Deprecated, use grant_revoke_privileges()
+  bool revoke_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1)
+  GrantRevokePrivilegeResponse grant_revoke_privileges(1:GrantRevokePrivilegeRequest request) throws(1:MetaException o1);
+
+  // this is used by metastore client to send UGI information to metastore server immediately
+  // after setting up a connection.
+  list<string> set_ugi(1:string user_name, 2:list<string> group_names) throws (1:MetaException o1)
+
+  //Authentication (delegation token) interfaces
+
+  // get metastore server delegation token for use from the map/reduce tasks to authenticate
+  // to metastore server
+  string get_delegation_token(1:string token_owner, 2:string renewer_kerberos_principal_name)
+    throws (1:MetaException o1)
+
+  // method to renew delegation token obtained from metastore server
+  i64 renew_delegation_token(1:string token_str_form) throws (1:MetaException o1)
+
+  // method to cancel delegation token obtained from metastore server
+  void cancel_delegation_token(1:string token_str_form) throws (1:MetaException o1)
+
+  // Transaction and lock management calls
+  // Get just list of open transactions
+  GetOpenTxnsResponse get_open_txns()
+  // Get list of open transactions with state (open, aborted)
+  GetOpenTxnsInfoResponse get_open_txns_info()
+  OpenTxnsResponse open_txns(1:OpenTxnRequest rqst)
+  void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1)
+  void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
+  LockResponse lock(1:LockRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
+  LockResponse check_lock(1:CheckLockRequest rqst)
+    throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2, 3:NoSuchLockException o3)
+  void unlock(1:UnlockRequest rqst) throws (1:NoSuchLockException o1, 2:TxnOpenException o2)
+  ShowLocksResponse show_locks(1:ShowLocksRequest rqst)
+  void heartbeat(1:HeartbeatRequest ids) throws (1:NoSuchLockException o1, 2:NoSuchTxnException o2, 3:TxnAbortedException o3)
+  HeartbeatTxnRangeResponse heartbeat_txn_range(1:HeartbeatTxnRangeRequest txns)
+  void compact(1:CompactionRequest rqst) 
+  ShowCompactResponse show_compact(1:ShowCompactRequest rqst)
+
+  // Notification logging calls
+  NotificationEventResponse get_next_notification(1:NotificationEventRequest rqst) 
+  CurrentNotificationEventId get_current_notificationEventId()
+}
+
+// * Note about the DDL_TIME: When creating or altering a table or a partition,
+// if the DDL_TIME is not set, the current time will be used.
+
+// For storing info about archived partitions in parameters
+
+// Whether the partition is archived
+const string IS_ARCHIVED = "is_archived",
+// The original location of the partition, before archiving. After archiving,
+// this directory will contain the archive. When the partition
+// is dropped, this directory will be deleted
+const string ORIGINAL_LOCATION = "original_location",
+
+// Whether or not the table is considered immutable - immutable tables can only be
+// overwritten or created if unpartitioned, or if partitioned, partitions inside them
+// can only be overwritten or created. Immutability supports write-once and replace
+// semantics, but not append.
+const string IS_IMMUTABLE = "immutable",
+
+// these should be needed only for backward compatibility with filestore
+const string META_TABLE_COLUMNS   = "columns",
+const string META_TABLE_COLUMN_TYPES   = "columns.types",
+const string BUCKET_FIELD_NAME    = "bucket_field_name",
+const string BUCKET_COUNT         = "bucket_count",
+const string FIELD_TO_DIMENSION   = "field_to_dimension",
+const string META_TABLE_NAME      = "name",
+const string META_TABLE_DB        = "db",
+const string META_TABLE_LOCATION  = "location",
+const string META_TABLE_SERDE     = "serde",
+const string META_TABLE_PARTITION_COLUMNS = "partition_columns",
+const string META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types",
+const string FILE_INPUT_FORMAT    = "file.inputformat",
+const string FILE_OUTPUT_FORMAT   = "file.outputformat",
+const string META_TABLE_STORAGE   = "storage_handler",
+const string TABLE_IS_TRANSACTIONAL = "transactional",
+const string TABLE_NO_AUTO_COMPACT = "no_auto_compaction",
+
+
diff --git a/cpp/src/arrow/dbi/hiveserver2/types.cc b/cpp/src/arrow/dbi/hiveserver2/types.cc
new file mode 100644
index 0000000..30b9fbc
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/types.cc
@@ -0,0 +1,45 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/types.h"
+
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/util/logging.h"
+
+namespace arrow {
+namespace hiveserver2 {
+
+const PrimitiveType* ColumnDesc::GetPrimitiveType() const {
+  return static_cast<PrimitiveType*>(type_.get());
+}
+
+const CharacterType* ColumnDesc::GetCharacterType() const {
+  DCHECK(type_->type_id() == ColumnType::TypeId::CHAR ||
+         type_->type_id() == ColumnType::TypeId::VARCHAR);
+  return static_cast<CharacterType*>(type_.get());
+}
+
+const DecimalType* ColumnDesc::GetDecimalType() const {
+  DCHECK(type_->type_id() == ColumnType::TypeId::DECIMAL);
+  return static_cast<DecimalType*>(type_.get());
+}
+
+std::string PrimitiveType::ToString() const { return TypeIdToString(type_id_); }
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/types.h b/cpp/src/arrow/dbi/hiveserver2/types.h
new file mode 100644
index 0000000..38cebcc
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/types.h
@@ -0,0 +1,131 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+
+namespace arrow {
+namespace hiveserver2 {
+
+// Represents a column's type.
+//
+// For now only PrimitiveType is implemented, as thase are the only types Impala will
+// currently return. In the future, nested types will be represented as other subclasses
+// of ColumnType containing ptrs to other ColumnTypes - for example, an ArrayType subclass
+// would contain a single ptr to another ColumnType representing the type of objects
+// stored in the array.
+class ColumnType {
+ public:
+  virtual ~ColumnType() = default;
+
+  // Maps directly to TTypeId in the HiveServer2 interface.
+  enum class TypeId {
+    BOOLEAN,
+    TINYINT,
+    SMALLINT,
+    INT,
+    BIGINT,
+    FLOAT,
+    DOUBLE,
+    STRING,
+    TIMESTAMP,
+    BINARY,
+    ARRAY,
+    MAP,
+    STRUCT,
+    UNION,
+    USER_DEFINED,
+    DECIMAL,
+    NULL_TYPE,
+    DATE,
+    VARCHAR,
+    CHAR,
+    INVALID,
+  };
+
+  virtual TypeId type_id() const = 0;
+  virtual std::string ToString() const = 0;
+};
+
+class PrimitiveType : public ColumnType {
+ public:
+  explicit PrimitiveType(const TypeId& type_id) : type_id_(type_id) {}
+
+  TypeId type_id() const override { return type_id_; }
+  std::string ToString() const override;
+
+ private:
+  const TypeId type_id_;
+};
+
+// Represents CHAR and VARCHAR types.
+class CharacterType : public PrimitiveType {
+ public:
+  CharacterType(const TypeId& type_id, int max_length)
+      : PrimitiveType(type_id), max_length_(max_length) {}
+
+  int max_length() const { return max_length_; }
+
+ private:
+  const int max_length_;
+};
+
+// Represents DECIMAL types.
+class DecimalType : public PrimitiveType {
+ public:
+  DecimalType(const TypeId& type_id, int precision, int scale)
+      : PrimitiveType(type_id), precision_(precision), scale_(scale) {}
+
+  int precision() const { return precision_; }
+  int scale() const { return scale_; }
+
+ private:
+  const int precision_;
+  const int scale_;
+};
+
+// Represents the metadata for a single column.
+class ColumnDesc {
+ public:
+  ColumnDesc(const std::string& column_name, std::unique_ptr<ColumnType> type,
+             int position, const std::string& comment)
+      : column_name_(column_name),
+        type_(move(type)),
+        position_(position),
+        comment_(comment) {}
+
+  const std::string& column_name() const { return column_name_; }
+  const ColumnType* type() const { return type_.get(); }
+  int position() const { return position_; }
+  const std::string& comment() const { return comment_; }
+
+  const PrimitiveType* GetPrimitiveType() const;
+  const CharacterType* GetCharacterType() const;
+  const DecimalType* GetDecimalType() const;
+
+ private:
+  const std::string column_name_;
+  std::unique_ptr<ColumnType> type_;
+  const int position_;
+  const std::string comment_;
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/util.cc b/cpp/src/arrow/dbi/hiveserver2/util.cc
new file mode 100644
index 0000000..8d8b593
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/util.cc
@@ -0,0 +1,251 @@
+// 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.
+
+#include "arrow/dbi/hiveserver2/util.h"
+
+#include <algorithm>
+#include <memory>
+#include <sstream>
+#include <vector>
+
+#include "arrow/dbi/hiveserver2/columnar-row-set.h"
+#include "arrow/dbi/hiveserver2/thrift-internal.h"
+
+#include "arrow/dbi/hiveserver2/TCLIService.h"
+#include "arrow/dbi/hiveserver2/TCLIService_types.h"
+
+#include "arrow/status.h"
+
+namespace hs2 = apache::hive::service::cli::thrift;
+using std::string;
+using std::unique_ptr;
+
+namespace arrow {
+namespace hiveserver2 {
+
+// PrintResults
+namespace {
+
+const char kNullSymbol[] = "NULL";
+const char kTrueSymbol[] = "true";
+const char kFalseSymbol[] = "false";
+
+struct PrintInfo {
+  // The PrintInfo takes ownership of the Column ptr.
+  PrintInfo(Column* c, size_t m) : column(c), max_size(m) {}
+
+  unique_ptr<Column> column;
+  size_t max_size;
+};
+
+// Adds a horizontal line of '-'s, with '+'s at the column breaks.
+static void AddTableBreak(std::ostream& out, std::vector<PrintInfo>* columns) {
+  for (size_t i = 0; i < columns->size(); ++i) {
+    out << "+";
+    for (size_t j = 0; j < (*columns)[i].max_size + 2; ++j) {
+      out << "-";
+    }
+  }
+  out << "+\n";
+}
+
+// Returns the number of spaces needed to display n, i.e. the number of digits n has,
+// plus 1 if n is negative.
+static size_t NumSpaces(int64_t n) {
+  if (n < 0) {
+    return 1 + NumSpaces(-n);
+  } else if (n < 10) {
+    return 1;
+  } else {
+    return 1 + NumSpaces(n / 10);
+  }
+}
+
+// Returns the max size needed to display a column of integer type.
+template <typename T>
+static size_t GetIntMaxSize(T* column, const string& column_name) {
+  size_t max_size = column_name.size();
+  for (int i = 0; i < column->length(); ++i) {
+    if (!column->IsNull(i)) {
+      max_size = std::max(max_size, NumSpaces(column->data()[i]));
+    } else {
+      max_size = std::max(max_size, sizeof(kNullSymbol));
+    }
+  }
+  return max_size;
+}
+
+}  // namespace
+
+void Util::PrintResults(const Operation* op, std::ostream& out) {
+  unique_ptr<ColumnarRowSet> results;
+  bool has_more_rows = true;
+  while (has_more_rows) {
+    Status s = op->Fetch(&results, &has_more_rows);
+    if (!s.ok()) {
+      out << s.ToString();
+      return;
+    }
+
+    std::vector<ColumnDesc> column_descs;
+    s = op->GetResultSetMetadata(&column_descs);
+
+    if (!s.ok()) {
+      out << s.ToString();
+      return;
+    } else if (column_descs.size() == 0) {
+      out << "No result set to print.\n";
+      return;
+    }
+
+    std::vector<PrintInfo> columns;
+    for (int i = 0; i < static_cast<int>(column_descs.size()); i++) {
+      const string column_name = column_descs[i].column_name();
+      switch (column_descs[i].type()->type_id()) {
+        case ColumnType::TypeId::BOOLEAN: {
+          BoolColumn* bool_col = results->GetBoolCol(i).release();
+
+          // The largest symbol is length 4 unless there is a FALSE, then is it
+          // kFalseSymbol.size() = 5.
+          size_t max_size = std::max(column_name.size(), sizeof(kTrueSymbol));
+          for (int j = 0; j < bool_col->length(); ++j) {
+            if (!bool_col->IsNull(j) && !bool_col->data()[j]) {
+              max_size = std::max(max_size, sizeof(kFalseSymbol));
+              break;
+            }
+          }
+
+          columns.emplace_back(bool_col, max_size);
+          break;
+        }
+        case ColumnType::TypeId::TINYINT: {
+          ByteColumn* byte_col = results->GetByteCol(i).release();
+          columns.emplace_back(byte_col, GetIntMaxSize(byte_col, column_name));
+          break;
+        }
+        case ColumnType::TypeId::SMALLINT: {
+          Int16Column* int16_col = results->GetInt16Col(i).release();
+          columns.emplace_back(int16_col, GetIntMaxSize(int16_col, column_name));
+          break;
+        }
+        case ColumnType::TypeId::INT: {
+          Int32Column* int32_col = results->GetInt32Col(i).release();
+          columns.emplace_back(int32_col, GetIntMaxSize(int32_col, column_name));
+          break;
+        }
+        case ColumnType::TypeId::BIGINT: {
+          Int64Column* int64_col = results->GetInt64Col(i).release();
+          columns.emplace_back(int64_col, GetIntMaxSize(int64_col, column_name));
+          break;
+        }
+        case ColumnType::TypeId::STRING: {
+          unique_ptr<StringColumn> string_col = results->GetStringCol(i);
+
+          size_t max_size = column_name.size();
+          for (int j = 0; j < string_col->length(); ++j) {
+            if (!string_col->IsNull(j)) {
+              max_size = std::max(max_size, string_col->data()[j].size());
+            } else {
+              max_size = std::max(max_size, sizeof(kNullSymbol));
+            }
+          }
+
+          columns.emplace_back(string_col.release(), max_size);
+          break;
+        }
+        case ColumnType::TypeId::BINARY:
+          columns.emplace_back(results->GetBinaryCol(i).release(), column_name.size());
+          break;
+        default: {
+          out << "Unrecognized ColumnType = " << column_descs[i].type()->ToString();
+        }
+      }
+    }
+
+    AddTableBreak(out, &columns);
+    for (size_t i = 0; i < columns.size(); ++i) {
+      out << "| " << column_descs[i].column_name() << " ";
+
+      int padding =
+          static_cast<int>(columns[i].max_size - column_descs[i].column_name().size());
+      while (padding > 0) {
+        out << " ";
+        --padding;
+      }
+    }
+    out << "|\n";
+    AddTableBreak(out, &columns);
+
+    for (int i = 0; i < columns[0].column->length(); ++i) {
+      for (size_t j = 0; j < columns.size(); ++j) {
+        std::stringstream value;
+
+        if (columns[j].column->IsNull(i)) {
+          value << kNullSymbol;
+        } else {
+          switch (column_descs[j].type()->type_id()) {
+            case ColumnType::TypeId::BOOLEAN:
+              if (reinterpret_cast<BoolColumn*>(columns[j].column.get())->data()[i]) {
+                value << kTrueSymbol;
+              } else {
+                value << kFalseSymbol;
+              }
+              break;
+            case ColumnType::TypeId::TINYINT:
+              // The cast prevents us from printing this as a char.
+              value << static_cast<int16_t>(
+                  reinterpret_cast<ByteColumn*>(columns[j].column.get())->data()[i]);
+              break;
+            case ColumnType::TypeId::SMALLINT:
+              value << reinterpret_cast<Int16Column*>(columns[j].column.get())->data()[i];
+              break;
+            case ColumnType::TypeId::INT:
+              value << reinterpret_cast<Int32Column*>(columns[j].column.get())->data()[i];
+              break;
+            case ColumnType::TypeId::BIGINT:
+              value << reinterpret_cast<Int64Column*>(columns[j].column.get())->data()[i];
+              break;
+            case ColumnType::TypeId::STRING:
+              value
+                  << reinterpret_cast<StringColumn*>(columns[j].column.get())->data()[i];
+              break;
+            case ColumnType::TypeId::BINARY:
+              value
+                  << reinterpret_cast<BinaryColumn*>(columns[j].column.get())->data()[i];
+              break;
+            default:
+              value << "unrecognized type";
+              break;
+          }
+        }
+
+        string value_str = value.str();
+        out << "| " << value_str << " ";
+        int padding = static_cast<int>(columns[j].max_size - value_str.size());
+        while (padding > 0) {
+          out << " ";
+          --padding;
+        }
+      }
+      out << "|\n";
+    }
+    AddTableBreak(out, &columns);
+  }
+}
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/dbi/hiveserver2/util.h b/cpp/src/arrow/dbi/hiveserver2/util.h
new file mode 100644
index 0000000..a17e7b2
--- /dev/null
+++ b/cpp/src/arrow/dbi/hiveserver2/util.h
@@ -0,0 +1,36 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+#include "arrow/dbi/hiveserver2/operation.h"
+
+namespace arrow {
+namespace hiveserver2 {
+
+// Utility functions. Intended primary for testing purposes - clients should not
+// rely on stability of the behavior or API of these functions.
+class Util {
+ public:
+  // Fetches the operation's results and returns them in a nicely formatted string.
+  static void PrintResults(const Operation* op, std::ostream& out);
+};
+
+}  // namespace hiveserver2
+}  // namespace arrow
diff --git a/cpp/src/arrow/status.h b/cpp/src/arrow/status.h
index 539b8d1..4df6cc5 100644
--- a/cpp/src/arrow/status.h
+++ b/cpp/src/arrow/status.h
@@ -84,6 +84,7 @@ enum class StatusCode : char {
   PlasmaObjectNonexistent = 21,
   PlasmaStoreFull = 22,
   PlasmaObjectAlreadySealed = 23,
+  StillExecuting = 24
 };
 
 #if defined(__clang__)
@@ -122,6 +123,9 @@ class ARROW_EXPORT Status {
   // Return a success status.
   static Status OK() { return Status(); }
 
+  // Return a success status with extra info
+  static Status OK(const std::string& msg) { return Status(StatusCode::OK, msg); }
+
   // Return error status of an appropriate type.
   static Status OutOfMemory(const std::string& msg) {
     return Status(StatusCode::OutOfMemory, msg);
@@ -175,6 +179,8 @@ class ARROW_EXPORT Status {
     return Status(StatusCode::PlasmaStoreFull, msg);
   }
 
+  static Status StillExecuting() { return Status(StatusCode::StillExecuting, ""); }
+
   // Returns true iff the status indicates success.
   bool ok() const { return (state_ == NULL); }
 
@@ -203,6 +209,8 @@ class ARROW_EXPORT Status {
   // An object is too large to fit into the plasma store.
   bool IsPlasmaStoreFull() const { return code() == StatusCode::PlasmaStoreFull; }
 
+  bool IsStillExecuting() const { return code() == StatusCode::StillExecuting; }
+
   // Return a string representation of this status suitable for printing.
   // Returns the string "OK" for success.
   std::string ToString() const;
diff --git a/cpp/thirdparty/download_dependencies.sh b/cpp/thirdparty/download_dependencies.sh
index ab8c0b2..1f0ffed 100755
--- a/cpp/thirdparty/download_dependencies.sh
+++ b/cpp/thirdparty/download_dependencies.sh
@@ -65,6 +65,8 @@ wget -c -O $_DST/grpc.tar.gz https://github.com/grpc/grpc/archive/v$GRPC_VERSION
 
 wget -c -O $_DST/orc.tar.gz https://github.com/apache/orc/archive/rel/release-$ORC_VERSION.tar.gz
 
+wget -c -O $_DST/thrift.tar.gz http://archive.apache.org/dist/thrift/${THRIFT_VERSION}/thrift-${THRIFT_VERSION}.tar.gz
+
 echo "
 # Environment variables for offline Arrow build
 export ARROW_BOOST_URL=$_DST/boost.tar.gz
@@ -81,4 +83,5 @@ export ARROW_ZSTD_URL=$_DST/zstd.tar.gz
 export ARROW_PROTOBUF_URL=$_DST/protobuf.tar.gz
 export ARROW_GRPC_URL=$_DST/grpc.tar.gz
 export ARROW_ORC_URL=$_DST/orc.tar.gz
+export ARROW_THRIFT_URL=$_DST/thrift.tar.gz
 "
diff --git a/cpp/thirdparty/versions.txt b/cpp/thirdparty/versions.txt
index 554c719..e7ea231 100644
--- a/cpp/thirdparty/versions.txt
+++ b/cpp/thirdparty/versions.txt
@@ -32,3 +32,4 @@ ZSTD_VERSION=1.2.0
 PROTOBUF_VERSION=2.6.0
 GRPC_VERSION=1.12.1
 ORC_VERSION=1.5.1
+THRIFT_VERSION=0.11.0
diff --git a/dev/docker-compose.yml b/dev/docker-compose.yml
index 8f20dde..c832cc3 100644
--- a/dev/docker-compose.yml
+++ b/dev/docker-compose.yml
@@ -36,6 +36,12 @@ services:
     links:
       - hdfs-namenode:hdfs-namenode
 
+  impala:
+    image: cpcloud86/impala:java8-1
+    ports:
+      - "21050"
+    hostname: impala
+
   hdfs_integration:
     links:
       - hdfs-namenode:hdfs-namenode
@@ -49,6 +55,14 @@ services:
     volumes:
      - ../..:/apache-arrow
 
+  hiveserver2:
+    links:
+      - impala
+    build:
+      context: hiveserver2
+    volumes:
+      - ../..:/apache-arrow
+
   spark_integration:
     build:
       context: spark_integration
diff --git a/dev/docker_common/Dockerfile.xenial.base b/dev/docker_common/Dockerfile.xenial.base
new file mode 100644
index 0000000..b2589de
--- /dev/null
+++ b/dev/docker_common/Dockerfile.xenial.base
@@ -0,0 +1,61 @@
+#
+# 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.
+#
+
+FROM ubuntu:16.04
+
+# Basic OS utilities
+RUN apt-get update \
+ && apt-get install -y \
+      wget \
+      git \
+      pkg-config \
+      build-essential \
+      software-properties-common \
+      ninja-build \
+ && apt-get clean
+
+ENV PATH="/opt/conda/bin:${PATH}"
+
+# install conda in /home/ubuntu/miniconda
+RUN wget https://repo.continuum.io/miniconda/Miniconda3-latest-Linux-x86_64.sh -O conda.sh \
+  && /bin/bash conda.sh -b -p /opt/conda \
+  && rm conda.sh \
+  && conda create -y -q -c conda-forge -n pyarrow-dev \
+      python=3.6 \
+      ipython \
+      nomkl \
+      numpy \
+      six \
+      setuptools \
+      cython \
+      pandas \
+      pytest \
+      cmake \
+      flatbuffers \
+      rapidjson \
+      boost-cpp \
+      thrift-cpp \
+      snappy \
+      zlib \
+      gflags \
+      brotli \
+      jemalloc \
+      lz4-c \
+      zstd \
+      setuptools \
+      setuptools_scm \
+ && conda clean --all
\ No newline at end of file
diff --git a/dev/docker_common/wait-for-it.sh b/dev/docker_common/wait-for-it.sh
new file mode 100755
index 0000000..51ce816
--- /dev/null
+++ b/dev/docker_common/wait-for-it.sh
@@ -0,0 +1,199 @@
+#!/usr/bin/env bash
+
+# The MIT License (MIT)
+# Copyright (c) 2016 Giles Hall
+#
+# Permission is hereby granted, free of charge, to any person obtaining a copy of
+# this software and associated documentation files (the "Software"), to deal in
+# the Software without restriction, including without limitation the rights to
+# use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+# of the Software, and to permit persons to whom the Software is furnished to do
+# so, subject to the following conditions:
+#
+# The above copyright notice and this permission notice shall be included in all
+# copies or substantial portions of the Software.
+#
+# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+# SOFTWARE.
+#
+#   Use this script to test if a given TCP host/port are available
+
+cmdname=$(basename $0)
+
+echoerr() { if [[ $QUIET -ne 1 ]]; then echo "$@" 1>&2; fi }
+
+usage()
+{
+    cat << USAGE >&2
+Usage:
+    $cmdname host:port [-s] [-t timeout] [-- command args]
+    -h HOST | --host=HOST       Host or IP under test
+    -p PORT | --port=PORT       TCP port under test
+                                Alternatively, you specify the host and port as host:port
+    -s | --strict               Only execute subcommand if the test succeeds
+    -q | --quiet                Don't output any status messages
+    -t TIMEOUT | --timeout=TIMEOUT
+                                Timeout in seconds, zero for no timeout
+    -- COMMAND ARGS             Execute command with args after the test finishes
+USAGE
+    exit 1
+}
+
+wait_for()
+{
+    if [[ $TIMEOUT -gt 0 ]]; then
+        echoerr "$cmdname: waiting $TIMEOUT seconds for $HOST:$PORT"
+    else
+        echoerr "$cmdname: waiting for $HOST:$PORT without a timeout"
+    fi
+    start_ts=$(date +%s)
+    while :
+    do
+        if [[ $ISBUSY -eq 1 ]]; then
+            nc -z $HOST $PORT
+            result=$?
+        else
+            (echo > /dev/tcp/$HOST/$PORT) >/dev/null 2>&1
+            result=$?
+        fi
+        if [[ $result -eq 0 ]]; then
+            end_ts=$(date +%s)
+            echoerr "$cmdname: $HOST:$PORT is available after $((end_ts - start_ts)) seconds"
+            break
+        fi
+        sleep 1
+    done
+    return $result
+}
+
+wait_for_wrapper()
+{
+    # In order to support SIGINT during timeout: http://unix.stackexchange.com/a/57692
+    if [[ $QUIET -eq 1 ]]; then
+        timeout $BUSYTIMEFLAG $TIMEOUT $0 --quiet --child --host=$HOST --port=$PORT --timeout=$TIMEOUT &
+    else
+        timeout $BUSYTIMEFLAG $TIMEOUT $0 --child --host=$HOST --port=$PORT --timeout=$TIMEOUT &
+    fi
+    PID=$!
+    trap "kill -INT -$PID" INT
+    wait $PID
+    RESULT=$?
+    if [[ $RESULT -ne 0 ]]; then
+        echoerr "$cmdname: timeout occurred after waiting $TIMEOUT seconds for $HOST:$PORT"
+    fi
+    return $RESULT
+}
+
+# process arguments
+while [[ $# -gt 0 ]]
+do
+    case "$1" in
+        *:* )
+        hostport=(${1//:/ })
+        HOST=${hostport[0]}
+        PORT=${hostport[1]}
+        shift 1
+        ;;
+        --child)
+        CHILD=1
+        shift 1
+        ;;
+        -q | --quiet)
+        QUIET=1
+        shift 1
+        ;;
+        -s | --strict)
+        STRICT=1
+        shift 1
+        ;;
+        -h)
+        HOST="$2"
+        if [[ $HOST == "" ]]; then break; fi
+        shift 2
+        ;;
+        --host=*)
+        HOST="${1#*=}"
+        shift 1
+        ;;
+        -p)
+        PORT="$2"
+        if [[ $PORT == "" ]]; then break; fi
+        shift 2
+        ;;
+        --port=*)
+        PORT="${1#*=}"
+        shift 1
+        ;;
+        -t)
+        TIMEOUT="$2"
+        if [[ $TIMEOUT == "" ]]; then break; fi
+        shift 2
+        ;;
+        --timeout=*)
+        TIMEOUT="${1#*=}"
+        shift 1
+        ;;
+        --)
+        shift
+        CLI=("$@")
+        break
+        ;;
+        --help)
+        usage
+        ;;
+        *)
+        echoerr "Unknown argument: $1"
+        usage
+        ;;
+    esac
+done
+
+if [[ "$HOST" == "" || "$PORT" == "" ]]; then
+    echoerr "Error: you need to provide a host and port to test."
+    usage
+fi
+
+TIMEOUT=${TIMEOUT:-15}
+STRICT=${STRICT:-0}
+CHILD=${CHILD:-0}
+QUIET=${QUIET:-0}
+
+# check to see if timeout is from busybox?
+# check to see if timeout is from busybox?
+TIMEOUT_PATH=$(realpath $(which timeout))
+if [[ $TIMEOUT_PATH =~ "busybox" ]]; then
+        ISBUSY=1
+        BUSYTIMEFLAG="-t"
+else
+        ISBUSY=0
+        BUSYTIMEFLAG=""
+fi
+
+if [[ $CHILD -gt 0 ]]; then
+    wait_for
+    RESULT=$?
+    exit $RESULT
+else
+    if [[ $TIMEOUT -gt 0 ]]; then
+        wait_for_wrapper
+        RESULT=$?
+    else
+        wait_for
+        RESULT=$?
+    fi
+fi
+
+if [[ $CLI != "" ]]; then
+    if [[ $RESULT -ne 0 && $STRICT -eq 1 ]]; then
+        echoerr "$cmdname: strict mode, refusing to execute subprocess"
+        exit $RESULT
+    fi
+    exec "${CLI[@]}"
+else
+    exit $RESULT
+fi
diff --git a/dev/hiveserver2/Dockerfile b/dev/hiveserver2/Dockerfile
new file mode 100644
index 0000000..36fa392
--- /dev/null
+++ b/dev/hiveserver2/Dockerfile
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+FROM arrow_integration_xenial_base
+
+ADD . /apache-arrow
+WORKDIR /apache-arrow
+
+CMD arrow/dev/hiveserver2/hiveserver2.sh
diff --git a/dev/hiveserver2/hiveserver2.sh b/dev/hiveserver2/hiveserver2.sh
new file mode 100755
index 0000000..0ff649e
--- /dev/null
+++ b/dev/hiveserver2/hiveserver2.sh
@@ -0,0 +1,67 @@
+#!/usr/bin/env bash
+#
+# 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.
+#
+
+# Exit on any error
+set -e
+
+# cwd is mounted from host machine to
+# and contains both arrow and parquet-cpp
+
+# Activate conda environment
+source activate pyarrow-dev
+
+# Arrow build variables
+export ARROW_BUILD_TYPE=debug
+export ARROW_BUILD_TOOLCHAIN=$CONDA_PREFIX
+export ARROW_HOME=$CONDA_PREFIX
+
+# For newer GCC per https://arrow.apache.org/docs/python/development.html#known-issues
+export CXXFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0"
+export PYARROW_CXXFLAGS=$CXXFLAGS
+export PYARROW_CMAKE_GENERATOR=Ninja
+
+_PWD=`pwd`
+ARROW_CPP_BUILD_DIR=$_PWD/arrow/cpp/hiveserver2-build
+DOCKER_COMMON_DIR=$_PWD/arrow/dev/docker_common
+
+function cleanup {
+    rm -rf $ARROW_CPP_BUILD_DIR
+}
+
+trap cleanup EXIT
+
+# Install arrow-cpp
+mkdir -p $ARROW_CPP_BUILD_DIR
+pushd $ARROW_CPP_BUILD_DIR
+
+cmake -GNinja \
+      -DCMAKE_BUILD_TYPE=$ARROW_BUILD_TYPE \
+      -DCMAKE_INSTALL_PREFIX=$ARROW_HOME \
+      -DARROW_HIVESERVER2=ON \
+      -DARROW_BUILD_TESTS=ON \
+      -DCMAKE_CXX_FLAGS=$CXXFLAGS \
+      ..
+ninja hiveserver2-test
+
+$DOCKER_COMMON_DIR/wait-for-it.sh impala:21050 -t 300 -s -- echo "impala is up"
+
+# Run C++ unit tests
+export ARROW_HIVESERVER2_TEST_HOST=impala
+debug/hiveserver2-test
+
+popd