You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/06/20 14:48:58 UTC

[impala] branch master updated: IMPALA-7290: part 2: Add HS2 support to Impala shell

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f1f3ae9  IMPALA-7290: part 2: Add HS2 support to Impala shell
f1f3ae9 is described below

commit f1f3ae9ec2b1f3e23ecfad7592e977647d692f7e
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Fri Mar 22 11:40:08 2019 -0700

    IMPALA-7290: part 2: Add HS2 support to Impala shell
    
    HS2 is added as an option via --protocol=hs2. The user-visible
    differences in behaviour are minimal. Beeswax is still the
    default and can be explicitly enabled via --protocol=beeswax
    but will be deprecated. The default is unchanged because
    changing the default could break certain workflows, e.g.
    those that explicitly specify the port with -i or deployments
    that hit --fe_service_threads for HS2 and somehow rely on
    impala-shell not contributing to that limit. For most
    workflows the change is transparent and we should change
    the default in a major version change.
    
    This support requires Impala-specific extensions to
    the HS2 interface, similar to the existing extensions
    to Beeswax. Thus the HS2 shell is only
    forwards-compatible with newer Impala versions.
    I considered trying to gracefully degrade when the
    new extensions weren't present, but it didn't seem to be
    worth the ongoing testing effort.
    
    Differences between HS2 and Beeswax are abstracted into
    ImpalaClient subclasses.
    Here are the changes required to make it work:
    * Switch to TBinaryProtocolAccelerated to avoid perf
      regression. The HS2 protocol requires decoding
      more primitive values (because its not a string-per-row),
      which was slow with the pure python implementation of
      TBinaryProtocol.
    * Added bitarray module to efficiently unpack null indicators
    * Minimise invasiveness of changes by transposing and stringifying
      the columnar results into rows in impala_client.py. The transposition
      needs to happen before display anyway.
    * Add PingImpalaHS2Service() to get back version string and webserver
      address.
    * Add CloseImpalaOperation() extension to return DML row counts. This
      possibly addresses IMPALA-1789, although we need to confirm that
      this is a sufficient solution.
    * Add is_closed member to query handles to avoid shell independently
      tracking whether the query handle was closed or not.
    * Include query status in HS2 log to match beeswax.
    * HS2 GetLog() command now includes query status error message for
      consistency with beeswax.
    * "set"/"set all" uses the client requests options, not the session
      default. This captures the effective value of TIMEZONE, which
      was previously missing. This also requires test changes where
      the tests set non-default values, e.g. for ABORT_ON_ERROR.
    * "set all" on the server side returns REMOVED query options - the
      shell needs to know these so it can correctly ignore them.
    * Clean up self.orig_cmd/self.last_leading comment argument
      passing to avoid implicit parameter passing through multiple
      function calls.
    * Clean up argument handling in shell tests to consistently pass
      around lists of arguments instead of strings that are subject
      to shell tokenisation rules.
    * Consistently close connections in the shell to avoid leaking
      HS2 sessions. This is enforced by making ImpalaShell a context
      manager and also eliminating all sys.exit() calls that would
      bypass the explicit connection closing.
    
    Testing:
    * Shell tests can run with both protocols
    * Add tests for formatting of all types and NULL values
    * Added testing for floating point output formatting, which does
      change as a result of switching to server-side vs client-side
      formatting.
    * Verified that newly-added tests were actually going through HS2
      by disabling hs2 on the minicluster and running tests.
    * Add checks to test_verify_metrics.py to ensure that no sessions
      are left open at the end of tests.
    
    Performance:
    Baseline from beeswax shell for large extract is as follows:
    
      $ time impala-shell.sh -B -q 'select * from tpch_parquet.orders' > /dev/null
      real    0m6.708s
      user    0m5.132s
      sys     0m0.204s
    
    After this change it is somewhat slower, but we generally don't consider
    bulk extract performance through the shell to be perf-critical:
      real    0m7.625s
      user    0m6.436s
      sys     0m0.256s
    
    Change-Id: I6d5cc83d545aacc659523f29b1d6feed672e2a12
    Reviewed-on: http://gerrit.cloudera.org:8080/12884
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 LICENSE.txt                                        |    1 +
 be/src/runtime/dml-exec-state.cc                   |    6 +-
 be/src/runtime/dml-exec-state.h                    |    6 +-
 be/src/service/client-request-state.cc             |   20 +-
 be/src/service/client-request-state.h              |   12 +-
 be/src/service/impala-beeswax-server.cc            |   20 +-
 be/src/service/impala-hs2-server.cc                |   51 +
 be/src/service/impala-server.cc                    |    6 +-
 be/src/service/impala-server.h                     |   23 +-
 bin/rat_exclude_files.txt                          |    1 +
 common/thrift/ImpalaService.thrift                 |   46 +-
 infra/python/deps/compiled-requirements.txt        |    2 +-
 shell/ext-py/bitarray-0.9.0/PKG-INFO               |  593 ++++
 shell/ext-py/bitarray-0.9.0/bitarray/__init__.py   |  116 +
 shell/ext-py/bitarray-0.9.0/bitarray/_bitarray.c   | 3162 ++++++++++++++++++++
 .../bitarray-0.9.0/bitarray/test_bitarray.py       | 2196 ++++++++++++++
 shell/ext-py/bitarray-0.9.0/setup.py               |   47 +
 shell/impala_client.py                             | 1000 +++++--
 shell/impala_shell.py                              |  410 +--
 shell/impala_shell_config_defaults.py              |    2 +-
 shell/make_shell_tarball.sh                        |    7 +-
 shell/option_parser.py                             |    4 +
 shell/thrift_sasl.py                               |    1 -
 .../functional-query/queries/QueryTest/set.test    |    6 +-
 tests/common/impala_service.py                     |    4 +-
 tests/common/impala_test_suite.py                  |   13 +-
 tests/custom_cluster/test_shell_interactive.py     |    4 +-
 .../test_shell_interactive_reconnect.py            |   15 +-
 tests/hs2/test_hs2.py                              |    7 +-
 tests/query_test/test_observability.py             |   16 +-
 tests/shell/test_shell_commandline.py              |   32 +-
 tests/shell/test_shell_interactive.py              |   39 +-
 tests/shell/util.py                                |   23 +-
 tests/verifiers/metric_verifier.py                 |    5 +-
 tests/verifiers/test_verify_metrics.py             |    6 +
 35 files changed, 7372 insertions(+), 530 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index b8c2c01..fcf2f04 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -399,6 +399,7 @@ www/DataTables* and www/datatables*: MIT license
 shell/pkg_resources.py: Python Software License V2
 Parts of be/src/runtime/string-search.h: Python Software License V2
 Parts of shell/impala_shell.py: Python Software License V2
+shell/ext-py/bit-array*: Python Software License V2
 
   Copyright (c) 2001 - 2016 Python Software Foundation; All Rights Reserved
 
diff --git a/be/src/runtime/dml-exec-state.cc b/be/src/runtime/dml-exec-state.cc
index 5ab1bce..4474bae 100644
--- a/be/src/runtime/dml-exec-state.cc
+++ b/be/src/runtime/dml-exec-state.cc
@@ -412,18 +412,18 @@ void DmlExecState::ToProto(DmlExecStatusPB* dml_status) {
   }
 }
 
-void DmlExecState::ToTInsertResult(TInsertResult* insert_result) {
+void DmlExecState::ToTDmlResult(TDmlResult* dml_result) {
   lock_guard<mutex> l(lock_);
   int64_t num_row_errors = 0;
   bool has_kudu_stats = false;
   for (const PartitionStatusMap::value_type& v: per_partition_status_) {
-    insert_result->rows_modified[v.first] = v.second.num_modified_rows();
+    dml_result->rows_modified[v.first] = v.second.num_modified_rows();
     if (v.second.has_stats() && v.second.stats().has_kudu_stats()) {
       has_kudu_stats = true;
     }
     num_row_errors += v.second.stats().kudu_stats().num_row_errors();
   }
-  if (has_kudu_stats) insert_result->__set_num_row_errors(num_row_errors);
+  if (has_kudu_stats) dml_result->__set_num_row_errors(num_row_errors);
 }
 
 void DmlExecState::AddPartition(
diff --git a/be/src/runtime/dml-exec-state.h b/be/src/runtime/dml-exec-state.h
index 5872b85..e601be4 100644
--- a/be/src/runtime/dml-exec-state.h
+++ b/be/src/runtime/dml-exec-state.h
@@ -32,7 +32,7 @@ namespace impala {
 class DmlExecStatusPB;
 class DmlPartitionStatusPB;
 class DmlStatsPB;
-class TInsertResult;
+class TDmlResult;
 class TFinalizeParams;
 class TUpdateCatalogRequest;
 class RuntimeProfile;
@@ -105,9 +105,9 @@ class DmlExecState {
   /// Serialize to protobuf and stores the result in 'dml_status'.
   void ToProto(DmlExecStatusPB* dml_status);
 
-  /// Populates 'insert_result' with PartitionStatusMap data, for Impala's extension of
+  /// Populates 'dml_result' with PartitionStatusMap data, for Impala's extension of
   /// Beeswax.
-  void ToTInsertResult(TInsertResult* insert_result);
+  void ToTDmlResult(TDmlResult* dml_result);
 
  private:
   /// protects all fields below
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 010157b..452a546 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -267,15 +267,15 @@ Status ClientRequestState::Exec(TExecRequest* exec_request) {
 
 void ClientRequestState::PopulateResultForSet(bool is_set_all) {
   map<string, string> config;
-  TQueryOptionsToMap(session_->QueryOptions(), &config);
+  TQueryOptionsToMap(query_options(), &config);
   vector<string> keys, values, levels;
   map<string, string>::const_iterator itr = config.begin();
   for (; itr != config.end(); ++itr) {
     const auto opt_level_id =
         parent_server_->query_option_levels_[itr->first];
-    if (opt_level_id == TQueryOptionLevel::REMOVED) continue;
     if (!is_set_all && (opt_level_id == TQueryOptionLevel::DEVELOPMENT ||
-                        opt_level_id == TQueryOptionLevel::DEPRECATED)) {
+                        opt_level_id == TQueryOptionLevel::DEPRECATED ||
+                        opt_level_id == TQueryOptionLevel::REMOVED)) {
       continue;
     }
     keys.push_back(itr->first);
@@ -1294,6 +1294,20 @@ void ClientRequestState::UpdateFilter(const TUpdateFilterParams& params) {
   coord_->UpdateFilter(params);
 }
 
+bool ClientRequestState::GetDmlStats(TDmlResult* dml_result, Status* query_status) {
+  lock_guard<mutex> l(lock_);
+  *query_status = query_status_;
+  if (!query_status->ok()) return false;
+  // Coord may be NULL for a SELECT with LIMIT 0.
+  // Note that when IMPALA-87 is fixed (INSERT without FROM clause) we might
+  // need to revisit this, since that might lead us to insert a row without a
+  // coordinator, depending on how we choose to drive the table sink.
+  Coordinator* coord = GetCoordinator();
+  if (coord == nullptr) return false;
+  coord->dml_exec_state()->ToTDmlResult(dml_result);
+  return true;
+}
+
 void ClientRequestState::UpdateEndTime() {
   // Update the query's end time only if it isn't set previously.
   if (end_time_us_.CompareAndSwap(0, UnixMicros())) {
diff --git a/be/src/service/client-request-state.h b/be/src/service/client-request-state.h
index 9347a08..56a901d 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -164,6 +164,12 @@ class ClientRequestState {
       const TRuntimeProfileForest& thrift_profiles) WARN_UNUSED_RESULT;
   void UpdateFilter(const TUpdateFilterParams& params);
 
+  /// Populate DML stats in 'dml_result' if this request succeeded.
+  /// Sets 'query_status' to the overall query status.
+  /// Return true if the result was set, otherwise return false.
+  /// Caller must not hold 'lock()'.
+  bool GetDmlStats(TDmlResult* dml_result, Status* query_status);
+
   ImpalaServer::SessionState* session() const { return session_.get(); }
 
   /// Queries are run and authorized on behalf of the effective_user.
@@ -523,9 +529,9 @@ protected:
   void UpdateOperationState(
       apache::hive::service::cli::thrift::TOperationState::type operation_state);
 
-  /// Gets the query options, their values and levels and populates the result set
-  /// with them. It covers the subset of options for 'SET' and all of them for
-  /// 'SET ALL'
+  /// Gets the query options, their levels and the values for this client request
+  /// and populates the result set with them. It covers the subset of options for
+  /// 'SET' and all of them for 'SET ALL'
   void PopulateResultForSet(bool is_set_all);
 };
 
diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index b897dc5..9e3198b 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -382,7 +382,7 @@ void ImpalaServer::Cancel(impala::TStatus& tstatus,
   tstatus.status_code = TErrorCode::OK;
 }
 
-void ImpalaServer::CloseInsert(TInsertResult& insert_result,
+void ImpalaServer::CloseInsert(TDmlResult& dml_result,
     const QueryHandle& query_handle) {
   ScopedSessionState session_handle(this);
   shared_ptr<SessionState> session;
@@ -393,7 +393,7 @@ void ImpalaServer::CloseInsert(TInsertResult& insert_result,
   VLOG_QUERY << "CloseInsert(): query_id=" << PrintId(query_id);
 
   // CloseInsertInternal() will validates that 'session' has access to 'query_id'.
-  Status status = CloseInsertInternal(session.get(), query_id, &insert_result);
+  Status status = CloseInsertInternal(session.get(), query_id, &dml_result);
   if (!status.ok()) {
     RaiseBeeswaxException(status.GetDetail(), SQLSTATE_GENERAL_ERROR);
   }
@@ -586,7 +586,7 @@ Status ImpalaServer::FetchInternal(ClientRequestState* request_state,
 }
 
 Status ImpalaServer::CloseInsertInternal(SessionState* session, const TUniqueId& query_id,
-    TInsertResult* insert_result) {
+    TDmlResult* dml_result) {
   shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
   if (UNLIKELY(request_state == nullptr)) {
     string err_msg = Substitute("Invalid query handle: $0", PrintId(query_id));
@@ -598,19 +598,7 @@ Status ImpalaServer::CloseInsertInternal(SessionState* session, const TUniqueId&
       CheckClientRequestSession(session, request_state->effective_user(), query_id));
 
   Status query_status;
-  {
-    lock_guard<mutex> l(*request_state->lock());
-    query_status = request_state->query_status();
-    if (query_status.ok()) {
-      // Coord may be NULL for a SELECT with LIMIT 0.
-      // Note that when IMPALA-87 is fixed (INSERT without FROM clause) we might
-      // need to revisit this, since that might lead us to insert a row without a
-      // coordinator, depending on how we choose to drive the table sink.
-      if (request_state->GetCoordinator() != nullptr) {
-        request_state->GetCoordinator()->dml_exec_state()->ToTInsertResult(insert_result);
-      }
-    }
-  }
+  request_state->GetDmlStats(dml_result, &query_status);
   RETURN_IF_ERROR(UnregisterQuery(query_id, true));
   return query_status;
 }
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 82be134..9717963 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -710,6 +710,15 @@ void ImpalaServer::CancelOperation(TCancelOperationResp& return_val,
 
 void ImpalaServer::CloseOperation(TCloseOperationResp& return_val,
     const TCloseOperationReq& request) {
+  TCloseImpalaOperationReq request2;
+  request2.operationHandle = request.operationHandle;
+  TCloseImpalaOperationResp tmp_resp;
+  CloseImpalaOperation(tmp_resp, request2);
+  return_val.status = tmp_resp.status;
+}
+
+void ImpalaServer::CloseImpalaOperation(TCloseImpalaOperationResp& return_val,
+    const TCloseImpalaOperationReq& request) {
   TUniqueId query_id;
   TUniqueId op_secret;
   HS2_RETURN_IF_ERROR(return_val, THandleIdentifierToTUniqueId(
@@ -728,6 +737,12 @@ void ImpalaServer::CloseOperation(TCloseOperationResp& return_val,
   HS2_RETURN_IF_ERROR(return_val,
       session_handle.WithSession(session_id, SecretArg::Operation(op_secret, query_id)),
       SQLSTATE_GENERAL_ERROR);
+  if (request_state->stmt_type() == TStmtType::DML) {
+    Status query_status;
+    if (request_state->GetDmlStats(&return_val.dml_result, &query_status)) {
+      return_val.__isset.dml_result = true;
+    }
+  }
 
   // TODO: use timeout to get rid of unwanted request_state.
   HS2_RETURN_IF_ERROR(return_val, UnregisterQuery(query_id, true),
@@ -863,6 +878,18 @@ void ImpalaServer::GetLog(TGetLogResp& return_val, const TGetLogReq& request) {
     // Report progress
     ss << coord->progress().ToString() << "\n";
   }
+  // Report the query status, if the query failed.
+  {
+    // Take the lock to ensure that if the client sees a query_state == EXCEPTION, it is
+    // guaranteed to see the error query_status.
+    lock_guard<mutex> l(*request_state->lock());
+    Status query_status = request_state->query_status();
+    DCHECK_EQ(request_state->operation_state() == TOperationState::ERROR_STATE,
+        !query_status.ok());
+    // If the query status is !ok, include the status error message at the top of the log.
+    if (!query_status.ok()) ss << query_status.GetDetail();
+  }
+
   // Report analysis errors
   ss << join(request_state->GetAnalysisWarnings(), "\n");
   // Report queuing reason if the admission controller queued the query.
@@ -991,4 +1018,28 @@ void ImpalaServer::AddSessionToConnection(
   }
   session->connections.insert(connection_id);
 }
+
+void ImpalaServer::PingImpalaHS2Service(TPingImpalaHS2ServiceResp& return_val,
+    const TPingImpalaHS2ServiceReq& req) {
+  VLOG_QUERY << "PingImpalaHS2Service(): request=" << ThriftDebugString(req);
+  TUniqueId session_id;
+  TUniqueId secret;
+  HS2_RETURN_IF_ERROR(return_val,
+      THandleIdentifierToTUniqueId(req.sessionHandle.sessionId, &session_id, &secret),
+      SQLSTATE_GENERAL_ERROR);
+  ScopedSessionState session_handle(this);
+  shared_ptr<SessionState> session;
+  HS2_RETURN_IF_ERROR(return_val,
+      session_handle.WithSession(session_id, SecretArg::Session(secret), &session),
+      SQLSTATE_GENERAL_ERROR);
+  if (session == NULL) {
+    HS2_RETURN_ERROR(return_val,
+        Substitute("Invalid session id: $0", PrintId(session_id)),
+        SQLSTATE_GENERAL_ERROR);
+  }
+
+  return_val.__set_version(GetVersionString(true));
+  return_val.__set_webserver_address(ExecEnv::GetInstance()->webserver()->Url());
+  VLOG_RPC << "PingImpalaHS2Service(): return_val=" << ThriftDebugString(return_val);
+}
 }
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 1c20fe6..bf040f9 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -1091,8 +1091,12 @@ void ImpalaServer::PrepareQueryContext(const TNetworkAddress& backend_addr,
   query_ctx->__set_pid(getpid());
   int64_t now_us = UnixMicros();
   const Timezone& utc_tz = TimezoneDatabase::GetUtcTimezone();
+  // Fill in query options with default timezone so it is visible in "SET" command,
+  // profiles, etc.
+  if (query_ctx->client_request.query_options.timezone.empty()) {
+    query_ctx->client_request.query_options.timezone = TimezoneDatabase::LocalZoneName();
+  }
   string local_tz_name = query_ctx->client_request.query_options.timezone;
-  if (local_tz_name.empty()) local_tz_name = TimezoneDatabase::LocalZoneName();
   const Timezone* local_tz = TimezoneDatabase::FindTimezone(local_tz_name);
   if (local_tz != nullptr) {
     LOG(INFO) << "Found local timezone \"" << local_tz_name << "\".";
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index c0971ba..7f334ef 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -59,7 +59,7 @@ class RowDescriptor;
 class TCatalogUpdate;
 class TPlanExecRequest;
 class TPlanExecParams;
-class TInsertResult;
+class TDmlResult;
 class TReportExecStatusArgs;
 class TReportExecStatusResult;
 class TNetworkAddress;
@@ -241,7 +241,7 @@ class ImpalaServer : public ImpalaServiceIf,
   /// ImpalaService rpcs: extensions over Beeswax (implemented in
   /// impala-beeswax-server.cc)
   virtual void Cancel(impala::TStatus& status, const beeswax::QueryHandle& query_id);
-  virtual void CloseInsert(impala::TInsertResult& insert_result,
+  virtual void CloseInsert(impala::TDmlResult& dml_result,
       const beeswax::QueryHandle& query_handle);
 
   /// Pings the Impala service and gets the server version string.
@@ -327,6 +327,17 @@ class ImpalaServer : public ImpalaServiceIf,
       apache::hive::service::cli::thrift::TRenewDelegationTokenResp& return_val,
       const apache::hive::service::cli::thrift::TRenewDelegationTokenReq& req);
 
+  // Extensions to HS2 implemented by ImpalaHiveServer2Service.
+
+  /// Pings the Impala service and gets the server version string.
+  virtual void PingImpalaHS2Service(TPingImpalaHS2ServiceResp& return_val,
+      const TPingImpalaHS2ServiceReq& req);
+
+  /// Closes an Impala operation and returns additional information about the closed
+  /// operation.
+  virtual void CloseImpalaOperation(
+      TCloseImpalaOperationResp& return_val, const TCloseImpalaOperationReq& request);
+
   /// ImpalaInternalService rpcs
   void UpdateFilter(TUpdateFilterResult& return_val, const TUpdateFilterParams& params);
 
@@ -875,12 +886,12 @@ class ImpalaServer : public ImpalaServiceIf,
   Status FetchInternal(ClientRequestState* request_state, bool start_over,
       int32_t fetch_size, beeswax::Results* query_results) WARN_UNUSED_RESULT;
 
-  /// Populate insert_result and clean up exec state. If the query
-  /// status is an error, insert_result is not populated and the status is returned.
-  /// 'session' is RPC client's session, used to check whether the insert can
+  /// Populate dml_result and clean up exec state. If the query
+  /// status is an error, dml_result is not populated and the status is returned.
+  /// 'session' is RPC client's session, used to check whether the DML can
   /// be closed via that session.
   Status CloseInsertInternal(SessionState* session, const TUniqueId& query_id,
-      TInsertResult* insert_result) WARN_UNUSED_RESULT;
+      TDmlResult* dml_result) WARN_UNUSED_RESULT;
 
   /// HiveServer2 private methods (implemented in impala-hs2-server.cc)
 
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 1cf051c..182b717 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -41,6 +41,7 @@ tests/comparison/leopard/static/js/bootstrap*
 shell/ext-py/prettytable-0.7.1/*
 shell/ext-py/sqlparse-0.1.19/*
 shell/ext-py/sasl-0.1.1/*
+shell/ext-py/bitarray-0.9.0/*
 www/d3.v3.min.js
 www/jquery/jquery-1.12.4.min.js
 tests/comparison/leopard/static/css/hljs.css
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index cf09785..1b746e2 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -399,8 +399,7 @@ enum TImpalaQueryOptions {
 }
 
 // The summary of a DML statement.
-// TODO: Rename to reflect that this is for all DML.
-struct TInsertResult {
+struct TDmlResult {
   // Number of modified rows per partition. Only applies to HDFS and Kudu 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.
@@ -433,6 +432,40 @@ struct TResetTableReq {
   2: required string table_name
 }
 
+// PingImpalaHS2Service() - ImpalaHiveServer2Service version.
+// Pings the Impala server to confirm that the server is alive and the session identified
+// by 'sessionHandle' is open. Returns metadata about the server. This exists separate
+// from the base HS2 GetInfo() methods because not all relevant metadata is accessible
+// through GetInfo().
+struct TPingImpalaHS2ServiceReq {
+  1: required TCLIService.TSessionHandle sessionHandle
+}
+
+struct TPingImpalaHS2ServiceResp {
+  1: required TCLIService.TStatus status
+
+  // The Impala service's version string.
+  2: optional string version
+
+  // The Impalad's webserver address.
+  3: optional string webserver_address
+}
+
+// CloseImpalaOperation()
+//
+// Extended version of CloseOperation() that, if the operation was a DML
+// operation, returns statistics about the operation.
+struct TCloseImpalaOperationReq {
+  1: required TCLIService.TOperationHandle operationHandle
+}
+
+struct TCloseImpalaOperationResp {
+  1: required TCLIService.TStatus status
+
+  // Populated if the operation was a DML operation.
+  2: optional TDmlResult dml_result
+}
+
 // 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)
@@ -459,7 +492,7 @@ service ImpalaService extends beeswax.BeeswaxService {
       throws(1:beeswax.BeeswaxException error);
 
   // Closes the query handle and return the result summary of the insert.
-  TInsertResult CloseInsert(1:beeswax.QueryHandle handle)
+  TDmlResult 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
@@ -510,4 +543,11 @@ service ImpalaHiveServer2Service extends TCLIService.TCLIService {
 
   // Returns the runtime profile string for the given query
   TGetRuntimeProfileResp GetRuntimeProfile(1:TGetRuntimeProfileReq req);
+
+  // Client calls this RPC to verify that the server is an ImpalaService. Returns the
+  // server version.
+  TPingImpalaHS2ServiceResp PingImpalaHS2Service(1:TPingImpalaHS2ServiceReq req);
+
+  // Same as HS2 CloseOperation but can return additional information.
+  TCloseImpalaOperationResp CloseImpalaOperation(1:TCloseImpalaOperationReq req);
 }
diff --git a/infra/python/deps/compiled-requirements.txt b/infra/python/deps/compiled-requirements.txt
index 0a406c2..457591a 100644
--- a/infra/python/deps/compiled-requirements.txt
+++ b/infra/python/deps/compiled-requirements.txt
@@ -20,7 +20,7 @@
 
 argparse == 1.4.0
 impyla == 0.15.0
-  bitarray == 0.8.1
+  bitarray == 0.9.0
   sasl == 0.1.3
   six == 1.11.0
   thrift-sasl == 0.1.0
diff --git a/shell/ext-py/bitarray-0.9.0/PKG-INFO b/shell/ext-py/bitarray-0.9.0/PKG-INFO
new file mode 100644
index 0000000..b867d86
--- /dev/null
+++ b/shell/ext-py/bitarray-0.9.0/PKG-INFO
@@ -0,0 +1,593 @@
+Metadata-Version: 1.1
+Name: bitarray
+Version: 0.9.0
+Summary: efficient arrays of booleans -- C extension
+Home-page: https://github.com/ilanschnell/bitarray
+Author: Ilan Schnell
+Author-email: ilanschnell@gmail.com
+License: PSF
+Description: ======================================
+        bitarray: efficient arrays of booleans
+        ======================================
+        
+        This module provides an object type which efficiently represents an array
+        of booleans.  Bitarrays are sequence types and behave very much like usual
+        lists.  Eight bits are represented by one byte in a contiguous block of
+        memory.  The user can select between two representations: little-endian
+        and big-endian.  All of the functionality is implemented in C.
+        Methods for accessing the machine representation are provided.
+        This can be useful when bit level access to binary files is required,
+        such as portable bitmap image files (.pbm).  Also, when dealing with
+        compressed data which uses variable bit length encoding, you may find
+        this module useful.
+        
+        
+        Key features
+        ------------
+        
+         * All functionality implemented in C.
+        
+         * Bitarray objects behave very much like a list object, in particular
+           slicing (including slice assignment and deletion) is supported.
+        
+         * The bit endianness can be specified for each bitarray object, see below.
+        
+         * On 32bit systems, a bitarray object can contain up to 2^34 elements,
+           that is 16 Gbits (on 64bit machines up to 2^63 elements in theory --
+           on Python 2.4 only 2^31 elements,
+           see `PEP 353 <http://www.python.org/dev/peps/pep-0353/>`_
+           (added in Python 2.5)).
+        
+         * Packing and unpacking to other binary data formats,
+           e.g. `numpy.ndarray <http://www.scipy.org/Tentative_NumPy_Tutorial>`_,
+           is possible.
+        
+         * Fast methods for encoding and decoding variable bit length prefix codes
+        
+         * Sequential search (as list or iterator)
+        
+         * Bitwise operations: ``&, |, ^, &=, |=, ^=, ~``
+        
+         * Pickling and unpickling of bitarray objects possible.
+        
+         * Bitarray objects support the buffer protocol (Python 2.7 only)
+        
+        
+        Installation
+        ------------
+        
+        bitarray can be installed from source::
+        
+           $ tar xzf bitarray-0.9.0.tar.gz
+           $ cd bitarray-0.9.0
+           $ python setup.py install
+        
+        On Unix systems, the latter command may have to be executed with root
+        privileges.
+        If you have `distribute <http://pypi.python.org/pypi/distribute/>`_
+        installed, you can easy_install bitarray.
+        Once you have installed the package, you may want to test it::
+        
+           $ python -c 'import bitarray; bitarray.test()'
+           bitarray is installed in: /usr/local/lib/python2.7/site-packages/bitarray
+           bitarray version: 0.9.0
+           2.7.2 (r271:86832, Nov 29 2010) [GCC 4.2.1 (SUSE Linux)]
+           .........................................................................
+           ...........................................
+           ----------------------------------------------------------------------
+           Ran 134 tests in 1.396s
+           
+           OK
+        
+        You can always import the function test,
+        and ``test().wasSuccessful()`` will return True when the test went well.
+        
+        
+        
+        Using the module
+        ----------------
+        
+        As mentioned above, bitarray objects behave very much like lists, so
+        there is not too much to learn.  The biggest difference from list objects
+        is the ability to access the machine representation of the object.
+        When doing so, the bit endianness is of importance; this issue is
+        explained in detail in the section below.  Here, we demonstrate the
+        basic usage of bitarray objects:
+        
+           >>> from bitarray import bitarray
+           >>> a = bitarray()            # create empty bitarray
+           >>> a.append(True)
+           >>> a.extend([False, True, True])
+           >>> a
+           bitarray('1011')
+        
+        Bitarray objects can be instantiated in different ways:
+        
+           >>> a = bitarray(2**20)       # bitarray of length 1048576 (uninitialized)
+           >>> bitarray('1001011')       # from a string
+           bitarray('1001011')
+           >>> lst = [True, False, False, True, False, True, True]
+           >>> bitarray(lst)             # from list, tuple, iterable
+           bitarray('1001011')
+        
+        Bits can be assigned from any Python object, if the value can be interpreted
+        as a truth value.  You can think of this as Python's built-in function bool()
+        being applied, whenever casting an object:
+        
+           >>> a = bitarray([42, '', True, {}, 'foo', None])
+           >>> a
+           bitarray('101010')
+           >>> a.append(a)      # note that bool(a) is True
+           >>> a.count(42)      # counts occurrences of True (not 42)
+           4L
+           >>> a.remove('')     # removes first occurrence of False
+           >>> a
+           bitarray('110101')
+        
+        Like lists, bitarray objects support slice assignment and deletion:
+        
+           >>> a = bitarray(50)
+           >>> a.setall(False)
+           >>> a[11:37:3] = 9 * bitarray([True])
+           >>> a
+           bitarray('00000000000100100100100100100100100100000000000000')
+           >>> del a[12::3]
+           >>> a
+           bitarray('0000000000010101010101010101000000000')
+           >>> a[-6:] = bitarray('10011')
+           >>> a
+           bitarray('000000000001010101010101010100010011')
+           >>> a += bitarray('000111')
+           >>> a[9:]
+           bitarray('001010101010101010100010011000111')
+        
+        In addition, slices can be assigned to booleans, which is easier (and
+        faster) than assigning to a bitarray in which all values are the same:
+        
+           >>> a = 20 * bitarray('0')
+           >>> a[1:15:3] = True
+           >>> a
+           bitarray('01001001001001000000')
+        
+        This is easier and faster than:
+        
+           >>> a = 20 * bitarray('0')
+           >>> a[1:15:3] = 5 * bitarray('1')
+           >>> a
+           bitarray('01001001001001000000')
+        
+        Note that in the latter we have to create a temporary bitarray whose length
+        must be known or calculated.
+        
+        
+        Bit endianness
+        --------------
+        
+        Since a bitarray allows addressing of individual bits, where the machine
+        represents 8 bits in one byte, there are two obvious choices for this
+        mapping: little- and big-endian.
+        When creating a new bitarray object, the endianness can always be
+        specified explicitly:
+        
+           >>> a = bitarray(endian='little')
+           >>> a.frombytes(b'A')
+           >>> a
+           bitarray('10000010')
+           >>> b = bitarray('11000010', endian='little')
+           >>> b.tobytes()
+           'C'
+        
+        Here, the low-bit comes first because little-endian means that increasing
+        numeric significance corresponds to an increasing address (index).
+        So a[0] is the lowest and least significant bit, and a[7] is the highest
+        and most significant bit.
+        
+           >>> a = bitarray(endian='big')
+           >>> a.frombytes(b'A')
+           >>> a
+           bitarray('01000001')
+           >>> a[6] = 1
+           >>> a.tobytes()
+           'C'
+        
+        Here, the high-bit comes first because big-endian
+        means "most-significant first".
+        So a[0] is now the lowest and most significant bit, and a[7] is the highest
+        and least significant bit.
+        
+        The bit endianness is a property attached to each bitarray object.
+        When comparing bitarray objects, the endianness (and hence the machine
+        representation) is irrelevant; what matters is the mapping from indices
+        to bits:
+        
+           >>> bitarray('11001', endian='big') == bitarray('11001', endian='little')
+           True
+        
+        Bitwise operations (``&, |, ^, &=, |=, ^=, ~``) are implemented efficiently
+        using the corresponding byte operations in C, i.e. the operators act on the
+        machine representation of the bitarray objects.  Therefore, one has to be
+        cautious when applying the operation to bitarrays with different endianness.
+        
+        When converting to and from machine representation, using
+        the ``tobytes``, ``frombytes``, ``tofile`` and ``fromfile`` methods,
+        the endianness matters:
+        
+           >>> a = bitarray(endian='little')
+           >>> a.frombytes(b'\x01')
+           >>> a
+           bitarray('10000000')
+           >>> b = bitarray(endian='big')
+           >>> b.frombytes(b'\x80')
+           >>> b
+           bitarray('10000000')
+           >>> a == b
+           True
+           >>> a.tobytes() == b.tobytes()
+           False
+        
+        The endianness can not be changed once an object is created.
+        However, since creating a bitarray from another bitarray just copies the
+        memory representing the data, you can create a new bitarray with different
+        endianness:
+        
+           >>> a = bitarray('11100000', endian='little')
+           >>> a
+           bitarray('11100000')
+           >>> b = bitarray(a, endian='big')
+           >>> b
+           bitarray('00000111')
+           >>> a == b
+           False
+           >>> a.tobytes() == b.tobytes()
+           True
+        
+        The default bit endianness is currently big-endian; however, this may change
+        in the future, and when dealing with the machine representation of bitarray
+        objects, it is recommended to always explicitly specify the endianness.
+        
+        Unless explicitly converting to machine representation, using
+        the ``tobytes``, ``frombytes``, ``tofile`` and ``fromfile`` methods,
+        the bit endianness will have no effect on any computation, and one
+        can safely ignore setting the endianness, and other details of this section.
+        
+        
+        Buffer protocol
+        ---------------
+        
+        Python 2.7 provides memoryview objects, which allow Python code to access
+        the internal data of an object that supports the buffer protocol without
+        copying.  Bitarray objects support this protocol, with the memory being
+        interpreted as simple bytes.
+        
+           >>> a = bitarray('01000001' '01000010' '01000011', endian='big')
+           >>> v = memoryview(a)
+           >>> len(v)
+           3
+           >>> v[-1]
+           'C'
+           >>> v[:2].tobytes()
+           'AB'
+           >>> v.readonly  # changing a bitarray's memory is also possible
+           False
+           >>> v[1] = 'o'
+           >>> a
+           bitarray('010000010110111101000011')
+        
+        
+        Variable bit length prefix codes
+        --------------------------------
+        
+        The method ``encode`` takes a dictionary mapping symbols to bitarrays
+        and an iterable, and extends the bitarray object with the encoded symbols
+        found while iterating.  For example:
+        
+           >>> d = {'H':bitarray('111'), 'e':bitarray('0'),
+           ...      'l':bitarray('110'), 'o':bitarray('10')}
+           ...
+           >>> a = bitarray()
+           >>> a.encode(d, 'Hello')
+           >>> a
+           bitarray('111011011010')
+        
+        Note that the string ``'Hello'`` is an iterable, but the symbols are not
+        limited to characters, in fact any immutable Python object can be a symbol.
+        Taking the same dictionary, we can apply the ``decode`` method which will
+        return a list of the symbols:
+        
+           >>> a.decode(d)
+           ['H', 'e', 'l', 'l', 'o']
+           >>> ''.join(a.decode(d))
+           'Hello'
+        
+        Since symbols are not limited to being characters, it is necessary to return
+        them as elements of a list, rather than simply returning the joined string.
+        
+        
+        Reference
+        ---------
+        
+        **The bitarray class:**
+        
+        ``bitarray([initial], [endian=string])``
+           Return a new bitarray object whose items are bits initialized from
+           the optional initial, and endianness.
+           If no object is provided, the bitarray is initialized to have length zero.
+           The initial object may be of the following types:
+           
+           int, long
+               Create bitarray of length given by the integer.  The initial values
+               in the array are random, because only the memory allocated.
+           
+           string
+               Create bitarray from a string of '0's and '1's.
+           
+           list, tuple, iterable
+               Create bitarray from a sequence, each element in the sequence is
+               converted to a bit using truth value value.
+           
+           bitarray
+               Create bitarray from another bitarray.  This is done by copying the
+               memory holding the bitarray data, and is hence very fast.
+           
+           The optional keyword arguments 'endian' specifies the bit endianness of the
+           created bitarray object.
+           Allowed values are 'big' and 'little' (default is 'big').
+           
+           Note that setting the bit endianness only has an effect when accessing the
+           machine representation of the bitarray, i.e. when using the methods: tofile,
+           fromfile, tobytes, frombytes.
+        
+        
+        **A bitarray object supports the following methods:**
+        
+        ``all()`` -> bool
+           Returns True when all bits in the array are True.
+        
+        
+        ``any()`` -> bool
+           Returns True when any bit in the array is True.
+        
+        
+        ``append(item)``
+           Append the value bool(item) to the end of the bitarray.
+        
+        
+        ``buffer_info()`` -> tuple
+           Return a tuple (address, size, endianness, unused, allocated) giving the
+           current memory address, the size (in bytes) used to hold the bitarray's
+           contents, the bit endianness as a string, the number of unused bits
+           (e.g. a bitarray of length 11 will have a buffer size of 2 bytes and
+           5 unused bits), and the size (in bytes) of the allocated memory.
+        
+        
+        ``bytereverse()``
+           For all bytes representing the bitarray, reverse the bit order (in-place).
+           Note: This method changes the actual machine values representing the
+           bitarray; it does not change the endianness of the bitarray object.
+        
+        
+        ``copy()`` -> bitarray
+           Return a copy of the bitarray.
+        
+        
+        ``count([value])`` -> int
+           Return number of occurrences of value (defaults to True) in the bitarray.
+        
+        
+        ``decode(code)`` -> list
+           Given a prefix code (a dict mapping symbols to bitarrays),
+           decode the content of the bitarray and return the list of symbols.
+        
+        
+        ``encode(code, iterable)``
+           Given a prefix code (a dict mapping symbols to bitarrays),
+           iterate over the iterable object with symbols, and extend the bitarray
+           with the corresponding bitarray for each symbols.
+        
+        
+        ``endian()`` -> string
+           Return the bit endianness as a string (either 'little' or 'big').
+        
+        
+        ``extend(object)``
+           Append bits to the end of the bitarray.  The objects which can be passed
+           to this method are the same iterable objects which can given to a bitarray
+           object upon initialization.
+        
+        
+        ``fill()`` -> int
+           Adds zeros to the end of the bitarray, such that the length of the bitarray
+           will be a multiple of 8.  Returns the number of bits added (0..7).
+        
+        
+        ``frombytes(bytes)``
+           Append from a byte string, interpreted as machine values.
+        
+        
+        ``fromfile(f, [n])``
+           Read n bytes from the file object f and append them to the bitarray
+           interpreted as machine values.  When n is omitted, as many bytes are
+           read until EOF is reached.
+        
+        
+        ``fromstring(string)``
+           Append from a string, interpreting the string as machine values.
+           Deprecated since version 0.4.0, use ``frombytes()`` instead.
+        
+        
+        ``index(value, [start, [stop]])`` -> int
+           Return index of the first occurrence of bool(value) in the bitarray.
+           Raises ValueError if the value is not present.
+        
+        
+        ``insert(i, item)``
+           Insert bool(item) into the bitarray before position i.
+        
+        
+        ``invert()``
+           Invert all bits in the array (in-place),
+           i.e. convert each 1-bit into a 0-bit and vice versa.
+        
+        
+        ``iterdecode(code)`` -> iterator
+           Given a prefix code (a dict mapping symbols to bitarrays),
+           decode the content of the bitarray and iterate over the symbols.
+        
+        
+        ``itersearch(bitarray)`` -> iterator
+           Searches for the given a bitarray in self, and return an iterator over
+           the start positions where bitarray matches self.
+        
+        
+        ``length()`` -> int
+           Return the length, i.e. number of bits stored in the bitarray.
+           This method is preferred over __len__ (used when typing ``len(a)``),
+           since __len__ will fail for a bitarray object with 2^31 or more elements
+           on a 32bit machine, whereas this method will return the correct value,
+           on 32bit and 64bit machines.
+        
+        
+        ``pack(bytes)``
+           Extend the bitarray from a byte string, where each characters corresponds to
+           a single bit.  The character b'\x00' maps to bit 0 and all other characters
+           map to bit 1.
+           This method, as well as the unpack method, are meant for efficient
+           transfer of data between bitarray objects to other python objects
+           (for example NumPy's ndarray object) which have a different view of memory.
+        
+        
+        ``pop([i])`` -> item
+           Return the i-th (default last) element and delete it from the bitarray.
+           Raises IndexError if bitarray is empty or index is out of range.
+        
+        
+        ``remove(item)``
+           Remove the first occurrence of bool(item) in the bitarray.
+           Raises ValueError if item is not present.
+        
+        
+        ``reverse()``
+           Reverse the order of bits in the array (in-place).
+        
+        
+        ``search(bitarray, [limit])`` -> list
+           Searches for the given a bitarray in self, and returns the start positions
+           where bitarray matches self as a list.
+           The optional argument limits the number of search results to the integer
+           specified.  By default, all search results are returned.
+        
+        
+        ``setall(value)``
+           Set all bits in the bitarray to bool(value).
+        
+        
+        ``sort(reverse=False)``
+           Sort the bits in the array (in-place).
+        
+        
+        ``to01()`` -> string
+           Return a string containing '0's and '1's, representing the bits in the
+           bitarray object.
+           Note: To extend a bitarray from a string containing '0's and '1's,
+           use the extend method.
+        
+        
+        ``tobytes()`` -> bytes
+           Return the byte representation of the bitarray.
+           When the length of the bitarray is not a multiple of 8, the few remaining
+           bits (1..7) are set to 0.
+        
+        
+        ``tofile(f)``
+           Write all bits (as machine values) to the file object f.
+           When the length of the bitarray is not a multiple of 8,
+           the remaining bits (1..7) are set to 0.
+        
+        
+        ``tolist()`` -> list
+           Return an ordinary list with the items in the bitarray.
+           Note that the list object being created will require 32 or 64 times more
+           memory than the bitarray object, which may cause a memory error if the
+           bitarray is very large.
+           Also note that to extend a bitarray with elements from a list,
+           use the extend method.
+        
+        
+        ``tostring()`` -> string
+           Return the string representing (machine values) of the bitarray.
+           When the length of the bitarray is not a multiple of 8, the few remaining
+           bits (1..7) are set to 0.
+           Deprecated since version 0.4.0, use ``tobytes()`` instead.
+        
+        
+        ``unpack(zero=b'\x00', one=b'\xff')`` -> bytes
+           Return a byte string containing one character for each bit in the bitarray,
+           using the specified mapping.
+           See also the pack method.
+        
+        
+        **Functions defined in the module:**
+        
+        ``test(verbosity=1, repeat=1)`` -> TextTestResult
+           Run self-test, and return unittest.runner.TextTestResult object.
+        
+        
+        ``bitdiff(a, b)`` -> int
+           Return the difference between two bitarrays a and b.
+           This is function does the same as (a ^ b).count(), but is more memory
+           efficient, as no intermediate bitarray object gets created
+        
+        
+        ``bits2bytes(n)`` -> int
+           Return the number of bytes necessary to store n bits.
+        
+        
+        Change log
+        ----------
+        
+        **0.9.0** (2019-04-22):
+        
+          * more efficient decode and iterdecode by using C-level binary tree
+            instead of a python one, #54
+          * added buffer protocol support for Python 3, #55
+          * fixed invalid pointer exceptions in pypy, #47
+          * made all examples Py3k compatible
+          * add gene sequence example
+          * add official Python 3.7 support
+          * drop Python 2.4, 3.1 and 3.2 support
+        
+        
+        **0.8.3** (2018-07-06):
+        
+          * add exception to setup.py when README.rst cannot be opened
+        
+        
+        **0.8.2** (2018-05-30):
+        
+          * add official Python 3.6 support (although it was already working)
+          * fix description of fill(), #52
+          * handle extending self correctly, #28
+          * copy_n: fast copy with memmove fixed, #43
+          * minor clarity/wording changes to README, #23
+        
+        
+        Please find the complete change log
+        `here <https://github.com/ilanschnell/bitarray/blob/master/CHANGE_LOG>`_.
+        
+Platform: UNKNOWN
+Classifier: License :: OSI Approved :: Python Software Foundation License
+Classifier: Development Status :: 5 - Production/Stable
+Classifier: Intended Audience :: Developers
+Classifier: Operating System :: OS Independent
+Classifier: Programming Language :: C
+Classifier: Programming Language :: Python :: 2
+Classifier: Programming Language :: Python :: 2.5
+Classifier: Programming Language :: Python :: 2.6
+Classifier: Programming Language :: Python :: 2.7
+Classifier: Programming Language :: Python :: 3
+Classifier: Programming Language :: Python :: 3.3
+Classifier: Programming Language :: Python :: 3.4
+Classifier: Programming Language :: Python :: 3.5
+Classifier: Programming Language :: Python :: 3.6
+Classifier: Programming Language :: Python :: 3.7
+Classifier: Topic :: Utilities
diff --git a/shell/ext-py/bitarray-0.9.0/bitarray/__init__.py b/shell/ext-py/bitarray-0.9.0/bitarray/__init__.py
new file mode 100644
index 0000000..6a97795
--- /dev/null
+++ b/shell/ext-py/bitarray-0.9.0/bitarray/__init__.py
@@ -0,0 +1,116 @@
+"""
+This package defines an object type which can efficiently represent
+a bitarray.  Bitarrays are sequence types and behave very much like lists.
+
+Please find a description of this package at:
+
+    http://pypi.python.org/pypi/bitarray/
+
+Author: Ilan Schnell
+"""
+from bitarray._bitarray import _bitarray, bitdiff, bits2bytes, _sysinfo
+
+__version__ = '0.9.0'
+
+
+def _check_codedict(codedict):
+    if not isinstance(codedict, dict):
+        raise TypeError("dictionary expected")
+    if len(codedict) == 0:
+        raise ValueError("prefix code empty")
+    for k, v in codedict.items():
+        if not isinstance(v, bitarray):
+            raise TypeError("bitarray expected for dictionary value")
+        if v.length() == 0:
+            raise ValueError("non-empty bitarray expected")
+
+
+class bitarray(_bitarray):
+    """bitarray([initial], [endian=string])
+
+Return a new bitarray object whose items are bits initialized from
+the optional initial, and endianness.
+If no object is provided, the bitarray is initialized to have length zero.
+The initial object may be of the following types:
+
+int, long
+    Create bitarray of length given by the integer.  The initial values
+    in the array are random, because only the memory allocated.
+
+string
+    Create bitarray from a string of '0's and '1's.
+
+list, tuple, iterable
+    Create bitarray from a sequence, each element in the sequence is
+    converted to a bit using truth value value.
+
+bitarray
+    Create bitarray from another bitarray.  This is done by copying the
+    memory holding the bitarray data, and is hence very fast.
+
+The optional keyword arguments 'endian' specifies the bit endianness of the
+created bitarray object.
+Allowed values are 'big' and 'little' (default is 'big').
+
+Note that setting the bit endianness only has an effect when accessing the
+machine representation of the bitarray, i.e. when using the methods: tofile,
+fromfile, tobytes, frombytes."""
+
+    def fromstring(self, string):
+        """fromstring(string)
+
+Append from a string, interpreting the string as machine values.
+Deprecated since version 0.4.0, use ``frombytes()`` instead."""
+        return self.frombytes(string.encode())
+
+    def tostring(self):
+        """tostring() -> string
+
+Return the string representing (machine values) of the bitarray.
+When the length of the bitarray is not a multiple of 8, the few remaining
+bits (1..7) are set to 0.
+Deprecated since version 0.4.0, use ``tobytes()`` instead."""
+        return self.tobytes().decode()
+
+    def decode(self, codedict):
+        """decode(code) -> list
+
+Given a prefix code (a dict mapping symbols to bitarrays),
+decode the content of the bitarray and return the list of symbols."""
+        _check_codedict(codedict)
+        return self._decode(codedict)
+
+    def iterdecode(self, codedict):
+        """iterdecode(code) -> iterator
+
+Given a prefix code (a dict mapping symbols to bitarrays),
+decode the content of the bitarray and iterate over the symbols."""
+        _check_codedict(codedict)
+        return self._iterdecode(codedict)
+
+    def encode(self, codedict, iterable):
+        """encode(code, iterable)
+
+Given a prefix code (a dict mapping symbols to bitarrays),
+iterate over the iterable object with symbols, and extend the bitarray
+with the corresponding bitarray for each symbols."""
+        _check_codedict(codedict)
+        self._encode(codedict, iterable)
+
+    def __int__(self):
+        raise TypeError("int() argument cannot be a bitarray")
+
+    def __long__(self):
+        raise TypeError("long() argument cannot be a bitarray")
+
+    def __float__(self):
+        raise TypeError("float() argument cannot be a bitarray")
+
+
+def test(verbosity=1, repeat=1):
+    """test(verbosity=1, repeat=1) -> TextTestResult
+
+Run self-test, and return unittest.runner.TextTestResult object.
+"""
+    from bitarray import test_bitarray
+    return test_bitarray.run(verbosity=verbosity, repeat=repeat)
diff --git a/shell/ext-py/bitarray-0.9.0/bitarray/_bitarray.c b/shell/ext-py/bitarray-0.9.0/bitarray/_bitarray.c
new file mode 100644
index 0000000..d605f4c
--- /dev/null
+++ b/shell/ext-py/bitarray-0.9.0/bitarray/_bitarray.c
@@ -0,0 +1,3162 @@
+/*
+   This file is the C part of the bitarray package.  Almost all
+   functionality is implemented here.
+
+   Author: Ilan Schnell
+*/
+
+#define PY_SSIZE_T_CLEAN
+#include "Python.h"
+
+#if PY_MAJOR_VERSION >= 3
+#define IS_PY3K
+#endif
+
+#ifdef IS_PY3K
+#include "bytesobject.h"
+#define PyString_FromStringAndSize  PyBytes_FromStringAndSize
+#define PyString_FromString  PyBytes_FromString
+#define PyString_Check  PyBytes_Check
+#define PyString_Size  PyBytes_Size
+#define PyString_AsString  PyBytes_AsString
+#define PyString_ConcatAndDel  PyBytes_ConcatAndDel
+#define Py_TPFLAGS_HAVE_WEAKREFS  0
+#endif
+
+#if PY_MAJOR_VERSION == 2 && PY_MINOR_VERSION < 6
+/* backward compatibility with Python 2.5 */
+#define Py_TYPE(ob)   (((PyObject *) (ob))->ob_type)
+#define Py_SIZE(ob)   (((PyVarObject *) (ob))->ob_size)
+#endif
+
+#if PY_MAJOR_VERSION == 3 || (PY_MAJOR_VERSION == 2 && PY_MINOR_VERSION == 7)
+/* (new) buffer protocol */
+#define WITH_BUFFER
+#endif
+
+#ifdef STDC_HEADERS
+#include <stddef.h>
+#else  /* !STDC_HEADERS */
+#ifdef HAVE_SYS_TYPES_H
+#include <sys/types.h>      /* For size_t */
+#endif /* HAVE_SYS_TYPES_H */
+#endif /* !STDC_HEADERS */
+
+
+typedef long long int idx_t;
+
+/* throughout:  0 = little endian   1 = big endian */
+#define DEFAULT_ENDIAN  1
+
+typedef struct {
+    PyObject_VAR_HEAD
+#ifdef WITH_BUFFER
+    int ob_exports;             /* how many buffer exports */
+#endif
+    char *ob_item;
+    Py_ssize_t allocated;       /* how many bytes allocated */
+    idx_t nbits;                /* length og bitarray */
+    int endian;                 /* bit endianness of bitarray */
+    PyObject *weakreflist;      /* list of weak references */
+} bitarrayobject;
+
+static PyTypeObject Bitarraytype;
+
+#define bitarray_Check(obj)  PyObject_TypeCheck(obj, &Bitarraytype)
+
+#define BITS(bytes)  (((idx_t) 8) * ((idx_t) (bytes)))
+
+#define BYTES(bits)  (((bits) == 0) ? 0 : (((bits) - 1) / 8 + 1))
+
+#define BITMASK(endian, i)  (((char) 1) << ((endian) ? (7 - (i)%8) : (i)%8))
+
+/* ------------ low level access to bits in bitarrayobject ------------- */
+
+#define GETBIT(self, i)  \
+    ((self)->ob_item[(i) / 8] & BITMASK((self)->endian, i) ? 1 : 0)
+
+static void
+setbit(bitarrayobject *self, idx_t i, int bit)
+{
+    char *cp, mask;
+
+    mask = BITMASK(self->endian, i);
+    cp = self->ob_item + i / 8;
+    if (bit)
+        *cp |= mask;
+    else
+        *cp &= ~mask;
+}
+
+static int
+check_overflow(idx_t nbits)
+{
+    idx_t max_bits;
+
+    assert(nbits >= 0);
+    if (sizeof(void *) == 4) {  /* 32bit machine */
+        max_bits = ((idx_t) 1) << 34;  /* 2^34 = 16 Gbits*/
+        if (nbits > max_bits) {
+            char buff[256];
+            sprintf(buff, "cannot create bitarray of size %lld, "
+                          "max size is %lld", nbits, max_bits);
+            PyErr_SetString(PyExc_OverflowError, buff);
+            return -1;
+        }
+    }
+    return 0;
+}
+
+static int
+resize(bitarrayobject *self, idx_t nbits)
+{
+    Py_ssize_t newsize;
+    size_t _new_size;       /* for allocation */
+
+    if (check_overflow(nbits) < 0)
+        return -1;
+
+    newsize = (Py_ssize_t) BYTES(nbits);
+
+    /* Bypass realloc() when a previous overallocation is large enough
+       to accommodate the newsize.  If the newsize is 16 smaller than the
+       current size, then proceed with the realloc() to shrink the list.
+    */
+    if (self->allocated >= newsize &&
+        Py_SIZE(self) < newsize + 16 &&
+        self->ob_item != NULL)
+    {
+        Py_SIZE(self) = newsize;
+        self->nbits = nbits;
+        return 0;
+    }
+
+    if (newsize >= Py_SIZE(self) + 65536)
+        /* Don't overallocate when the size increase is very large. */
+        _new_size = newsize;
+    else
+        /* This over-allocates proportional to the bitarray size, making
+           room for additional growth.  The over-allocation is mild, but is
+           enough to give linear-time amortized behavior over a long
+           sequence of appends() in the presence of a poorly-performing
+           system realloc().
+           The growth pattern is:  0, 4, 8, 16, 25, 34, 44, 54, 65, 77, ...
+           Note, the pattern starts out the same as for lists but then
+           grows at a smaller rate so that larger bitarrays only overallocate
+           by about 1/16th -- this is done because bitarrays are assumed
+           to be memory critical.
+        */
+        _new_size = (newsize >> 4) + (Py_SIZE(self) < 8 ? 3 : 7) + newsize;
+
+    self->ob_item = PyMem_Realloc(self->ob_item, _new_size);
+    if (self->ob_item == NULL) {
+        PyErr_NoMemory();
+        return -1;
+    }
+    Py_SIZE(self) = newsize;
+    self->allocated = _new_size;
+    self->nbits = nbits;
+    return 0;
+}
+
+/* create new bitarray object without initialization of buffer */
+static PyObject *
+newbitarrayobject(PyTypeObject *type, idx_t nbits, int endian)
+{
+    bitarrayobject *obj;
+    Py_ssize_t nbytes;
+
+    if (check_overflow(nbits) < 0)
+        return NULL;
+
+    obj = (bitarrayobject *) type->tp_alloc(type, 0);
+    if (obj == NULL)
+        return NULL;
+
+    nbytes = (Py_ssize_t) BYTES(nbits);
+    Py_SIZE(obj) = nbytes;
+    obj->nbits = nbits;
+    obj->endian = endian;
+    if (nbytes == 0) {
+        obj->ob_item = NULL;
+    }
+    else {
+        obj->ob_item = PyMem_Malloc((size_t) nbytes);
+        if (obj->ob_item == NULL) {
+            PyObject_Del(obj);
+            PyErr_NoMemory();
+            return NULL;
+        }
+    }
+    obj->allocated = nbytes;
+    obj->weakreflist = NULL;
+    return (PyObject *) obj;
+}
+
+static void
+bitarray_dealloc(bitarrayobject *self)
+{
+    if (self->weakreflist != NULL)
+        PyObject_ClearWeakRefs((PyObject *) self);
+
+    if (self->ob_item != NULL)
+        PyMem_Free((void *) self->ob_item);
+
+    Py_TYPE(self)->tp_free((PyObject *) self);
+}
+
+/* copy n bits from other (starting at b) onto self (starting at a) */
+static void
+copy_n(bitarrayobject *self, idx_t a,
+       bitarrayobject *other, idx_t b, idx_t n)
+{
+    idx_t i;
+
+    assert(0 <= n && n <= self->nbits && n <= other->nbits);
+    assert(0 <= a && a <= self->nbits - n);
+    assert(0 <= b && b <= other->nbits - n);
+    if (n == 0) {
+        return;
+    }
+
+    if (self->endian == other->endian && a % 8 == 0 && b % 8 == 0 && n >= 8)
+    {
+        const Py_ssize_t bytes = (Py_ssize_t) n / 8;
+        const idx_t bits = bytes * 8;
+
+        if (a <= b) {
+            memmove(self->ob_item + a / 8, other->ob_item + b / 8, bytes);
+        }
+        if (n != bits) {
+            copy_n(self, bits + a, other, bits + b, n - bits);
+        }
+        if (a > b) {
+            memmove(self->ob_item + a / 8, other->ob_item + b / 8, bytes);
+        }
+        return;
+    }
+
+    /* the different type of looping is only relevant when other and self
+       are the same object, i.e. when copying a piece of an bitarrayobject
+       onto itself */
+    if (a <= b) {
+        for (i = 0; i < n; i++)             /* loop forward (delete) */
+            setbit(self, i + a, GETBIT(other, i + b));
+    }
+    else {
+        for (i = n - 1; i >= 0; i--)      /* loop backwards (insert) */
+            setbit(self, i + a, GETBIT(other, i + b));
+    }
+}
+
+/* starting at start, delete n bits from self */
+static int
+delete_n(bitarrayobject *self, idx_t start, idx_t n)
+{
+    assert(0 <= start && start <= self->nbits);
+    assert(0 <= n && n <= self->nbits - start);
+    if (n == 0)
+        return 0;
+
+    copy_n(self, start, self, start + n, self->nbits - start - n);
+    return resize(self, self->nbits - n);
+}
+
+/* starting at start, insert n (uninitialized) bits into self */
+static int
+insert_n(bitarrayobject *self, idx_t start, idx_t n)
+{
+    assert(0 <= start && start <= self->nbits);
+    assert(n >= 0);
+    if (n == 0)
+        return 0;
+
+    if (resize(self, self->nbits + n) < 0)
+        return -1;
+    copy_n(self, start + n, self, start, self->nbits - start - n);
+    return 0;
+}
+
+/* sets ususet bits to 0, i.e. the ones in the last byte (if any),
+   and return the number of bits set -- self->nbits is unchanged */
+static int
+setunused(bitarrayobject *self)
+{
+    idx_t i, n;
+    int res = 0;
+
+    n = BITS(Py_SIZE(self));
+    for (i = self->nbits; i < n; i++) {
+        setbit(self, i, 0);
+        res++;
+    }
+    assert(res < 8);
+    return res;
+}
+
+/* repeat self n times */
+static int
+repeat(bitarrayobject *self, idx_t n)
+{
+    idx_t nbits, i;
+
+    if (n <= 0) {
+        if (resize(self, 0) < 0)
+            return -1;
+    }
+    if (n > 1) {
+        nbits = self->nbits;
+        if (resize(self, nbits * n) < 0)
+            return -1;
+        for (i = 1; i < n; i++)
+            copy_n(self, i * nbits, self, 0, nbits);
+    }
+    return 0;
+}
+
+
+enum op_type {
+    OP_and,
+    OP_or,
+    OP_xor,
+};
+
+/* perform bitwise operation */
+static int
+bitwise(bitarrayobject *self, PyObject *arg, enum op_type oper)
+{
+    bitarrayobject *other;
+    Py_ssize_t i;
+
+    if (!bitarray_Check(arg)) {
+        PyErr_SetString(PyExc_TypeError,
+                        "bitarray object expected for bitwise operation");
+        return -1;
+    }
+    other = (bitarrayobject *) arg;
+    if (self->nbits != other->nbits) {
+        PyErr_SetString(PyExc_ValueError,
+               "bitarrays of equal length expected for bitwise operation");
+        return -1;
+    }
+    setunused(self);
+    setunused(other);
+    switch (oper) {
+    case OP_and:
+        for (i = 0; i < Py_SIZE(self); i++)
+            self->ob_item[i] &= other->ob_item[i];
+        break;
+    case OP_or:
+        for (i = 0; i < Py_SIZE(self); i++)
+            self->ob_item[i] |= other->ob_item[i];
+        break;
+    case OP_xor:
+        for (i = 0; i < Py_SIZE(self); i++)
+            self->ob_item[i] ^= other->ob_item[i];
+        break;
+    }
+    return 0;
+}
+
+/* set the bits from start to stop (excluding) in self to val */
+static void
+setrange(bitarrayobject *self, idx_t start, idx_t stop, int val)
+{
+    idx_t i;
+
+    assert(0 <= start && start <= self->nbits);
+    assert(0 <= stop && stop <= self->nbits);
+    for (i = start; i < stop; i++)
+        setbit(self, i, val);
+}
+
+static void
+invert(bitarrayobject *self)
+{
+    Py_ssize_t i;
+
+    for (i = 0; i < Py_SIZE(self); i++)
+        self->ob_item[i] = ~self->ob_item[i];
+}
+
+/* reverse the order of bits in each byte of the buffer */
+static void
+bytereverse(bitarrayobject *self)
+{
+    static char trans[256];
+    static int setup = 0;
+    Py_ssize_t i;
+    unsigned char c;
+
+    if (!setup) {
+        /* setup translation table, which maps each byte to it's reversed:
+           trans = {0, 128, 64, 192, 32, 160, ..., 255} */
+        int j, k;
+        for (k = 0; k < 256; k++) {
+            trans[k] = 0x00;
+            for (j = 0; j < 8; j++)
+                if (1 << (7 - j) & k)
+                    trans[k] |= 1 << j;
+        }
+        setup = 1;
+    }
+
+    setunused(self);
+    for (i = 0; i < Py_SIZE(self); i++) {
+        c = self->ob_item[i];
+        self->ob_item[i] = trans[c];
+    }
+}
+
+
+static int bitcount_lookup[256] = {
+    0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
+    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
+    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
+    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
+    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
+    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
+    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
+    4, 5, 5, 6, 5, 6, 6, 7, 5, 6, 6, 7, 6, 7, 7, 8,
+};
+
+/* returns number of 1 bits */
+static idx_t
+count(bitarrayobject *self)
+{
+    Py_ssize_t i;
+    idx_t res = 0;
+    unsigned char c;
+
+    setunused(self);
+    for (i = 0; i < Py_SIZE(self); i++) {
+        c = self->ob_item[i];
+        res += bitcount_lookup[c];
+    }
+    return res;
+}
+
+/* return index of first occurrence of vi, -1 when x is not in found. */
+static idx_t
+findfirst(bitarrayobject *self, int vi, idx_t start, idx_t stop)
+{
+    Py_ssize_t j;
+    idx_t i;
+    char c;
+
+    if (Py_SIZE(self) == 0)
+        return -1;
+    if (start < 0 || start > self->nbits)
+        start = 0;
+    if (stop < 0 || stop > self->nbits)
+        stop = self->nbits;
+    if (start >= stop)
+        return -1;
+
+    if (stop > start + 8) {
+        /* seraching for 1 means: break when byte is not 0x00
+           searching for 0 means: break when byte is not 0xff */
+        c = vi ? 0x00 : 0xff;
+
+        /* skip ahead by checking whole bytes */
+        for (j = (Py_ssize_t) (start / 8); j < BYTES(stop); j++)
+            if (c ^ self->ob_item[j])
+                break;
+
+        if (j == Py_SIZE(self))
+            j--;
+        assert(0 <= j && j < Py_SIZE(self));
+
+        if (start < BITS(j))
+            start = BITS(j);
+    }
+
+    /* fine grained search */
+    for (i = start; i < stop; i++)
+        if (GETBIT(self, i) == vi)
+            return i;
+
+    return -1;
+}
+
+/* search for the first occurrence bitarray xa (in self), starting at p,
+   and return its position (-1 when not found)
+*/
+static idx_t
+search(bitarrayobject *self, bitarrayobject *xa, idx_t p)
+{
+    idx_t i;
+
+    assert(p >= 0);
+    while (p < self->nbits - xa->nbits + 1) {
+        for (i = 0; i < xa->nbits; i++)
+            if (GETBIT(self, p + i) != GETBIT(xa, i))
+                goto next;
+
+        return p;
+    next:
+        p++;
+    }
+    return -1;
+}
+
+static int
+set_item(bitarrayobject *self, idx_t i, PyObject *v)
+{
+    long vi;
+
+    assert(0 <= i && i < self->nbits);
+    vi = PyObject_IsTrue(v);
+    if (vi < 0)
+        return -1;
+    setbit(self, i, vi);
+    return 0;
+}
+
+static int
+append_item(bitarrayobject *self, PyObject *item)
+{
+    if (resize(self, self->nbits + 1) < 0)
+        return -1;
+    return set_item(self, self->nbits - 1, item);
+}
+
+static PyObject *
+unpack(bitarrayobject *self, char zero, char one)
+{
+    PyObject *res;
+    Py_ssize_t i;
+    char *str;
+
+    if (self->nbits > PY_SSIZE_T_MAX) {
+        PyErr_SetString(PyExc_OverflowError, "bitarray too large to unpack");
+        return NULL;
+    }
+    str = PyMem_Malloc((size_t) self->nbits);
+    if (str == NULL) {
+        PyErr_NoMemory();
+        return NULL;
+    }
+    for (i = 0; i < self->nbits; i++) {
+        *(str + i) = GETBIT(self, i) ? one : zero;
+    }
+    res = PyString_FromStringAndSize(str, (Py_ssize_t) self->nbits);
+    PyMem_Free((void *) str);
+    return res;
+}
+
+static int
+extend_bitarray(bitarrayobject *self, bitarrayobject *other)
+{
+    idx_t n_sum;
+    idx_t n_other_bits;
+
+    if (other->nbits == 0)
+        return 0;
+
+    /*
+        Note: other may be self. Thus we take the size before we resize,
+        ensuring we only copy the right parts of the array.
+    */
+    n_other_bits = other->nbits;
+    n_sum = self->nbits + other->nbits;
+
+    if (resize(self, n_sum) < 0)
+        return -1;
+
+    copy_n(self, n_sum - n_other_bits, other, 0, n_other_bits);
+    return 0;
+}
+
+static int
+extend_iter(bitarrayobject *self, PyObject *iter)
+{
+    PyObject *item;
+
+    assert(PyIter_Check(iter));
+    while ((item = PyIter_Next(iter)) != NULL) {
+        if (append_item(self, item) < 0) {
+            Py_DECREF(item);
+            return -1;
+        }
+        Py_DECREF(item);
+    }
+    if (PyErr_Occurred())
+        return -1;
+
+    return 0;
+}
+
+static int
+extend_list(bitarrayobject *self, PyObject *list)
+{
+    PyObject *item;
+    Py_ssize_t n, i;
+
+    assert(PyList_Check(list));
+    n = PyList_Size(list);
+    if (n == 0)
+        return 0;
+
+    if (resize(self, self->nbits + n) < 0)
+        return -1;
+
+    for (i = 0; i < n; i++) {
+        item = PyList_GetItem(list, i);
+        if (item == NULL)
+            return -1;
+        if (set_item(self, self->nbits - n + i, item) < 0)
+            return -1;
+    }
+    return 0;
+}
+
+static int
+extend_tuple(bitarrayobject *self, PyObject *tuple)
+{
+    PyObject *item;
+    Py_ssize_t n, i;
+
+    assert(PyTuple_Check(tuple));
+    n = PyTuple_Size(tuple);
+    if (n == 0)
+        return 0;
+
+    if (resize(self, self->nbits + n) < 0)
+        return -1;
+
+    for (i = 0; i < n; i++) {
+        item = PyTuple_GetItem(tuple, i);
+        if (item == NULL)
+            return -1;
+        if (set_item(self, self->nbits - n + i, item) < 0)
+            return -1;
+    }
+    return 0;
+}
+
+/* extend_string(): extend the bitarray from a string, where each whole
+   characters is converted to a single bit
+*/
+enum conv_tp {
+    STR_01,    /*  '0' -> 0    '1'  -> 1   no other characters allowed */
+    STR_RAW,   /*  0x00 -> 0   other -> 1                              */
+};
+
+static int
+extend_string(bitarrayobject *self, PyObject *string, enum conv_tp conv)
+{
+    Py_ssize_t strlen, i;
+    char c, *str;
+    int vi = 0;
+
+    assert(PyString_Check(string));
+    strlen = PyString_Size(string);
+    if (strlen == 0)
+        return 0;
+
+    if (resize(self, self->nbits + strlen) < 0)
+        return -1;
+
+    str = PyString_AsString(string);
+
+    for (i = 0; i < strlen; i++) {
+        c = *(str + i);
+        /* depending on conv, map c to bit */
+        switch (conv) {
+        case STR_01:
+            switch (c) {
+            case '0': vi = 0; break;
+            case '1': vi = 1; break;
+            default:
+                PyErr_Format(PyExc_ValueError,
+                             "character must be '0' or '1', found '%c'", c);
+                return -1;
+            }
+            break;
+        case STR_RAW:
+            vi = c ? 1 : 0;
+            break;
+        }
+        setbit(self, self->nbits - strlen + i, vi);
+    }
+    return 0;
+}
+
+static int
+extend_rawstring(bitarrayobject *self, PyObject *string)
+{
+    Py_ssize_t strlen;
+    char *str;
+
+    assert(PyString_Check(string) && self->nbits % 8 == 0);
+    strlen = PyString_Size(string);
+    if (strlen == 0)
+        return 0;
+
+    if (resize(self, self->nbits + BITS(strlen)) < 0)
+        return -1;
+
+    str = PyString_AsString(string);
+    memcpy(self->ob_item + (Py_SIZE(self) - strlen), str, strlen);
+    return 0;
+}
+
+static int
+extend_dispatch(bitarrayobject *self, PyObject *obj)
+{
+    PyObject *iter;
+    int ret;
+
+    /* dispatch on type */
+    if (bitarray_Check(obj))                              /* bitarray */
+        return extend_bitarray(self, (bitarrayobject *) obj);
+
+    if (PyList_Check(obj))                                    /* list */
+        return extend_list(self, obj);
+
+    if (PyTuple_Check(obj))                                  /* tuple */
+        return extend_tuple(self, obj);
+
+    if (PyString_Check(obj))                                 /* str01 */
+        return extend_string(self, obj, STR_01);
+
+#ifdef IS_PY3K
+    if (PyUnicode_Check(obj)) {                               /* str01 */
+        PyObject *string;
+        string = PyUnicode_AsEncodedString(obj, NULL, NULL);
+        ret = extend_string(self, string, STR_01);
+        Py_DECREF(string);
+        return ret;
+    }
+#endif
+
+    if (PyIter_Check(obj))                                    /* iter */
+        return extend_iter(self, obj);
+
+    /* finally, try to get the iterator of the object */
+    iter = PyObject_GetIter(obj);
+    if (iter == NULL) {
+        PyErr_SetString(PyExc_TypeError, "could not extend bitarray");
+        return -1;
+    }
+    ret = extend_iter(self, iter);
+    Py_DECREF(iter);
+    return ret;
+}
+
+/* --------- helper functions NOT involving bitarrayobjects ------------ */
+
+#define ENDIAN_STR(ba)  (((ba)->endian) ? "big" : "little")
+
+#ifdef IS_PY3K
+#define IS_INDEX(x)  (PyLong_Check(x) || PyIndex_Check(x))
+#define IS_INT_OR_BOOL(x)  (PyBool_Check(x) || PyLong_Check(x))
+#else
+#define IS_INDEX(x)  (PyInt_Check(x) || PyLong_Check(x) || PyIndex_Check(x))
+#define IS_INT_OR_BOOL(x)  (PyBool_Check(x) || PyInt_Check(x) || \
+                                               PyLong_Check(x))
+#endif
+
+/* given an PyLong (which must be 0 or 1), or a PyBool, return 0 or 1,
+   or -1 on error */
+static int
+IntBool_AsInt(PyObject *v)
+{
+    long x;
+
+    if (PyBool_Check(v))
+        return PyObject_IsTrue(v);
+
+#ifndef IS_PY3K
+    if (PyInt_Check(v)) {
+        x = PyInt_AsLong(v);
+    }
+    else
+#endif
+    if (PyLong_Check(v)) {
+        x = PyLong_AsLong(v);
+    }
+    else {
+        PyErr_SetString(PyExc_TypeError, "integer or bool expected");
+        return -1;
+    }
+
+    if (x < 0 || x > 1) {
+        PyErr_SetString(PyExc_ValueError,
+                        "integer value between 0 and 1 expected");
+        return -1;
+    }
+    return (int) x;
+}
+
+/* Extract a slice index from a PyInt or PyLong or an object with the
+   nb_index slot defined, and store in *i.
+   However, this function returns -1 on error and 0 on success.
+
+   This is almost _PyEval_SliceIndex() with Py_ssize_t replaced by idx_t
+*/
+static int
+getIndex(PyObject *v, idx_t *i)
+{
+    idx_t x;
+
+#ifndef IS_PY3K
+    if (PyInt_Check(v)) {
+        x = PyInt_AS_LONG(v);
+    }
+    else
+#endif
+    if (PyLong_Check(v)) {
+        x = PyLong_AsLongLong(v);
+    }
+    else if (PyIndex_Check(v)) {
+        x = PyNumber_AsSsize_t(v, NULL);
+        if (x == -1 && PyErr_Occurred())
+            return -1;
+    }
+    else {
+        PyErr_SetString(PyExc_TypeError, "slice indices must be integers or "
+                                         "None or have an __index__ method");
+        return -1;
+    }
+    *i = x;
+    return 0;
+}
+
+/* this is PySlice_GetIndicesEx() with Py_ssize_t replaced by idx_t */
+static int
+slice_GetIndicesEx(PySliceObject *r, idx_t length,
+                   idx_t *start, idx_t *stop, idx_t *step, idx_t *slicelength)
+{
+    idx_t defstart, defstop;
+
+    if (r->step == Py_None) {
+        *step = 1;
+    }
+    else {
+        if (getIndex(r->step, step) < 0)
+            return -1;
+        if (*step == 0) {
+            PyErr_SetString(PyExc_ValueError, "slice step cannot be zero");
+            return -1;
+        }
+    }
+    defstart = *step < 0 ? length - 1 : 0;
+    defstop = *step < 0 ? -1 : length;
+
+    if (r->start == Py_None) {
+        *start = defstart;
+    }
+    else {
+        if (getIndex(r->start, start) < 0)
+            return -1;
+        if (*start < 0) *start += length;
+        if (*start < 0) *start = (*step < 0) ? -1 : 0;
+        if (*start >= length) *start = (*step < 0) ? length - 1 : length;
+    }
+
+    if (r->stop == Py_None) {
+        *stop = defstop;
+    }
+    else {
+        if (getIndex(r->stop, stop) < 0)
+            return -1;
+        if (*stop < 0) *stop += length;
+        if (*stop < 0) *stop = -1;
+        if (*stop > length) *stop = length;
+    }
+
+    if ((*step < 0 && *stop >= *start) || (*step > 0 && *start >= *stop)) {
+        *slicelength = 0;
+    }
+    else if (*step < 0) {
+        *slicelength = (*stop - *start + 1) / (*step) + 1;
+    }
+    else {
+        *slicelength = (*stop - *start - 1) / (*step) + 1;
+    }
+
+    return 0;
+}
+
+/**************************************************************************
+                         Implementation of API methods
+ **************************************************************************/
+
+static PyObject *
+bitarray_length(bitarrayobject *self)
+{
+    return PyLong_FromLongLong(self->nbits);
+}
+
+PyDoc_STRVAR(length_doc,
+"length() -> int\n\
+\n\
+Return the length, i.e. number of bits stored in the bitarray.\n\
+This method is preferred over __len__ (used when typing ``len(a)``),\n\
+since __len__ will fail for a bitarray object with 2^31 or more elements\n\
+on a 32bit machine, whereas this method will return the correct value,\n\
+on 32bit and 64bit machines.");
+
+PyDoc_STRVAR(len_doc,
+"__len__() -> int\n\
+\n\
+Return the length, i.e. number of bits stored in the bitarray.\n\
+This method will fail for a bitarray object with 2^31 or more elements\n\
+on a 32bit machine.  Use bitarray.length() instead.");
+
+
+static PyObject *
+bitarray_copy(bitarrayobject *self)
+{
+    PyObject *res;
+
+    res = newbitarrayobject(Py_TYPE(self), self->nbits, self->endian);
+    if (res == NULL)
+        return NULL;
+
+    memcpy(((bitarrayobject *) res)->ob_item, self->ob_item, Py_SIZE(self));
+    return res;
+}
+
+PyDoc_STRVAR(copy_doc,
+"copy() -> bitarray\n\
+\n\
+Return a copy of the bitarray.");
+
+
+static PyObject *
+bitarray_count(bitarrayobject *self, PyObject *args)
+{
+    idx_t n1;
+    long x = 1;
+
+    if (!PyArg_ParseTuple(args, "|i:count", &x))
+        return NULL;
+
+    n1 = count(self);
+    return PyLong_FromLongLong(x ? n1 : (self->nbits - n1));
+}
+
+PyDoc_STRVAR(count_doc,
+"count([value]) -> int\n\
+\n\
+Return number of occurrences of value (defaults to True) in the bitarray.");
+
+
+static PyObject *
+bitarray_index(bitarrayobject *self, PyObject *args)
+{
+    PyObject *x;
+    idx_t i, start = 0, stop = -1;
+    long vi;
+
+    if (!PyArg_ParseTuple(args, "O|LL:index", &x, &start, &stop))
+        return NULL;
+
+    vi = PyObject_IsTrue(x);
+    if (vi < 0)
+        return NULL;
+
+    i = findfirst(self, vi, start, stop);
+    if (i < 0) {
+        PyErr_SetString(PyExc_ValueError, "index(x): x not in bitarray");
+        return NULL;
+    }
+    return PyLong_FromLongLong(i);
+}
+
+PyDoc_STRVAR(index_doc,
+"index(value, [start, [stop]]) -> int\n\
+\n\
+Return index of the first occurrence of bool(value) in the bitarray.\n\
+Raises ValueError if the value is not present.");
+
+
+static PyObject *
+bitarray_extend(bitarrayobject *self, PyObject *obj)
+{
+    if (extend_dispatch(self, obj) < 0)
+        return NULL;
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(extend_doc,
+"extend(object)\n\
+\n\
+Append bits to the end of the bitarray.  The objects which can be passed\n\
+to this method are the same iterable objects which can given to a bitarray\n\
+object upon initialization.");
+
+
+static PyObject *
+bitarray_contains(bitarrayobject *self, PyObject *x)
+{
+    long res;
+
+    if (IS_INT_OR_BOOL(x)) {
+        int vi;
+
+        vi = IntBool_AsInt(x);
+        if (vi < 0)
+            return NULL;
+        res = findfirst(self, vi, 0, -1) >= 0;
+    }
+    else if (bitarray_Check(x)) {
+        res = search(self, (bitarrayobject *) x, 0) >= 0;
+    }
+    else {
+        PyErr_SetString(PyExc_TypeError, "bitarray or bool expected");
+        return NULL;
+    }
+    return PyBool_FromLong(res);
+}
+
+PyDoc_STRVAR(contains_doc,
+"__contains__(x) -> bool\n\
+\n\
+Return True if bitarray contains x, False otherwise.\n\
+The value x may be a boolean (or integer between 0 and 1), or a bitarray.");
+
+
+static PyObject *
+bitarray_search(bitarrayobject *self, PyObject *args)
+{
+    PyObject *list = NULL;   /* list of matching positions to be returned */
+    PyObject *x, *item = NULL;
+    Py_ssize_t limit = -1;
+    bitarrayobject *xa;
+    idx_t p;
+
+    if (!PyArg_ParseTuple(args, "O|n:_search", &x, &limit))
+        return NULL;
+
+    if (!bitarray_Check(x)) {
+        PyErr_SetString(PyExc_TypeError, "bitarray expected for search");
+        return NULL;
+    }
+    xa = (bitarrayobject *) x;
+    if (xa->nbits == 0) {
+        PyErr_SetString(PyExc_ValueError, "can't search for empty bitarray");
+        return NULL;
+    }
+    list = PyList_New(0);
+    if (list == NULL)
+        return NULL;
+    if (xa->nbits > self->nbits || limit == 0)
+        return list;
+
+    p = 0;
+    while (1) {
+        p = search(self, xa, p);
+        if (p < 0)
+            break;
+        item = PyLong_FromLongLong(p);
+        p++;
+        if (item == NULL || PyList_Append(list, item) < 0) {
+            Py_XDECREF(item);
+            Py_XDECREF(list);
+            return NULL;
+        }
+        Py_DECREF(item);
+        if (limit > 0 && PyList_Size(list) >= limit)
+            break;
+    }
+    return list;
+}
+
+PyDoc_STRVAR(search_doc,
+"search(bitarray, [limit]) -> list\n\
+\n\
+Searches for the given a bitarray in self, and returns the start positions\n\
+where bitarray matches self as a list.\n\
+The optional argument limits the number of search results to the integer\n\
+specified.  By default, all search results are returned.");
+
+
+static PyObject *
+bitarray_buffer_info(bitarrayobject *self)
+{
+    PyObject *res, *ptr;
+
+    ptr = PyLong_FromVoidPtr(self->ob_item),
+    res = Py_BuildValue("OLsiL",
+                        ptr,
+                        (idx_t) Py_SIZE(self),
+                        ENDIAN_STR(self),
+                        (int) (BITS(Py_SIZE(self)) - self->nbits),
+                        (idx_t) self->allocated);
+    Py_DECREF(ptr);
+    return res;
+}
+
+PyDoc_STRVAR(buffer_info_doc,
+"buffer_info() -> tuple\n\
+\n\
+Return a tuple (address, size, endianness, unused, allocated) giving the\n\
+current memory address, the size (in bytes) used to hold the bitarray's\n\
+contents, the bit endianness as a string, the number of unused bits\n\
+(e.g. a bitarray of length 11 will have a buffer size of 2 bytes and\n\
+5 unused bits), and the size (in bytes) of the allocated memory.");
+
+
+static PyObject *
+bitarray_endian(bitarrayobject *self)
+{
+#ifdef IS_PY3K
+    return PyUnicode_FromString(ENDIAN_STR(self));
+#else
+    return PyString_FromString(ENDIAN_STR(self));
+#endif
+}
+
+PyDoc_STRVAR(endian_doc,
+"endian() -> string\n\
+\n\
+Return the bit endianness as a string (either 'little' or 'big').");
+
+
+static PyObject *
+bitarray_append(bitarrayobject *self, PyObject *v)
+{
+    if (append_item(self, v) < 0)
+        return NULL;
+
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(append_doc,
+"append(item)\n\
+\n\
+Append the value bool(item) to the end of the bitarray.");
+
+
+static PyObject *
+bitarray_all(bitarrayobject *self)
+{
+    if (findfirst(self, 0, 0, -1) >= 0)
+        Py_RETURN_FALSE;
+    else
+        Py_RETURN_TRUE;
+}
+
+PyDoc_STRVAR(all_doc,
+"all() -> bool\n\
+\n\
+Returns True when all bits in the array are True.");
+
+
+static PyObject *
+bitarray_any(bitarrayobject *self)
+{
+    if (findfirst(self, 1, 0, -1) >= 0)
+        Py_RETURN_TRUE;
+    else
+        Py_RETURN_FALSE;
+}
+
+PyDoc_STRVAR(any_doc,
+"any() -> bool\n\
+\n\
+Returns True when any bit in the array is True.");
+
+
+static PyObject *
+bitarray_reduce(bitarrayobject *self)
+{
+    PyObject *dict, *repr = NULL, *result = NULL;
+    char *str;
+
+    dict = PyObject_GetAttrString((PyObject *) self, "__dict__");
+    if (dict == NULL) {
+        PyErr_Clear();
+        dict = Py_None;
+        Py_INCREF(dict);
+    }
+    /* the first byte indicates the number of unused bits at the end, and
+       the rest of the bytes consist of the raw binary data */
+    str = PyMem_Malloc(Py_SIZE(self) + 1);
+    if (str == NULL) {
+        PyErr_NoMemory();
+        goto error;
+    }
+    str[0] = (char) setunused(self);
+    memcpy(str + 1, self->ob_item, Py_SIZE(self));
+    repr = PyString_FromStringAndSize(str, Py_SIZE(self) + 1);
+    if (repr == NULL)
+        goto error;
+    PyMem_Free((void *) str);
+    result = Py_BuildValue("O(Os)O", Py_TYPE(self),
+                           repr, ENDIAN_STR(self), dict);
+error:
+    Py_DECREF(dict);
+    Py_XDECREF(repr);
+    return result;
+}
+
+PyDoc_STRVAR(reduce_doc, "state information for pickling");
+
+
+static PyObject *
+bitarray_reverse(bitarrayobject *self)
+{
+    PyObject *t;    /* temp bitarray to store lower half of self */
+    idx_t i, m;
+
+    if (self->nbits < 2)
+        Py_RETURN_NONE;
+
+    t = newbitarrayobject(Py_TYPE(self), self->nbits / 2, self->endian);
+    if (t == NULL)
+        return NULL;
+
+#define tt  ((bitarrayobject *) t)
+    /* copy lower half of array into temporary array */
+    memcpy(tt->ob_item, self->ob_item, Py_SIZE(tt));
+
+    m = self->nbits - 1;
+
+    /* reverse the upper half onto the lower half. */
+    for (i = 0; i < tt->nbits; i++)
+        setbit(self, i, GETBIT(self, m - i));
+
+    /* revert the stored away lower half onto the upper half. */
+    for (i = 0; i < tt->nbits; i++)
+        setbit(self, m - i, GETBIT(tt, i));
+#undef tt
+    Py_DECREF(t);
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(reverse_doc,
+"reverse()\n\
+\n\
+Reverse the order of bits in the array (in-place).");
+
+
+static PyObject *
+bitarray_fill(bitarrayobject *self)
+{
+    long p;
+
+    p = setunused(self);
+    self->nbits += p;
+#ifdef IS_PY3K
+    return PyLong_FromLong(p);
+#else
+    return PyInt_FromLong(p);
+#endif
+}
+
+PyDoc_STRVAR(fill_doc,
+"fill() -> int\n\
+\n\
+Adds zeros to the end of the bitarray, such that the length of the bitarray\n\
+will be a multiple of 8.  Returns the number of bits added (0..7).");
+
+
+static PyObject *
+bitarray_invert(bitarrayobject *self)
+{
+    invert(self);
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(invert_doc,
+"invert()\n\
+\n\
+Invert all bits in the array (in-place),\n\
+i.e. convert each 1-bit into a 0-bit and vice versa.");
+
+
+static PyObject *
+bitarray_bytereverse(bitarrayobject *self)
+{
+    bytereverse(self);
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(bytereverse_doc,
+"bytereverse()\n\
+\n\
+For all bytes representing the bitarray, reverse the bit order (in-place).\n\
+Note: This method changes the actual machine values representing the\n\
+bitarray; it does not change the endianness of the bitarray object.");
+
+
+static PyObject *
+bitarray_setall(bitarrayobject *self, PyObject *v)
+{
+    long vi;
+
+    vi = PyObject_IsTrue(v);
+    if (vi < 0)
+        return NULL;
+
+    memset(self->ob_item, vi ? 0xff : 0x00, Py_SIZE(self));
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(setall_doc,
+"setall(value)\n\
+\n\
+Set all bits in the bitarray to bool(value).");
+
+
+static PyObject *
+bitarray_sort(bitarrayobject *self, PyObject *args, PyObject *kwds)
+{
+    idx_t n, n0, n1;
+    int reverse = 0;
+    static char* kwlist[] = {"reverse", NULL};
+
+    if (!PyArg_ParseTupleAndKeywords(args, kwds, "|i:sort", kwlist, &reverse))
+        return NULL;
+
+    n = self->nbits;
+    n1 = count(self);
+
+    if (reverse) {
+        setrange(self, 0, n1, 1);
+        setrange(self, n1, n, 0);
+    }
+    else {
+        n0 = n - n1;
+        setrange(self, 0, n0, 0);
+        setrange(self, n0, n, 1);
+    }
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(sort_doc,
+"sort(reverse=False)\n\
+\n\
+Sort the bits in the array (in-place).");
+
+
+#ifdef IS_PY3K
+static PyObject *
+bitarray_fromfile(bitarrayobject *self, PyObject *args)
+{
+    PyObject *f;
+    Py_ssize_t newsize, nbytes = -1;
+    PyObject *reader, *rargs, *result;
+    size_t nread;
+    idx_t t, p;
+
+    if (!PyArg_ParseTuple(args, "O|n:fromfile", &f, &nbytes))
+        return NULL;
+
+    if (nbytes == 0)
+        Py_RETURN_NONE;
+
+    reader = PyObject_GetAttrString(f, "read");
+    if (reader == NULL)
+    {
+        PyErr_SetString(PyExc_TypeError,
+                        "first argument must be an open file");
+        return NULL;
+    }
+    rargs = Py_BuildValue("(n)", nbytes);
+    if (rargs == NULL) {
+        Py_DECREF(reader);
+        return NULL;
+    }
+    result = PyEval_CallObject(reader, rargs);
+    if (result != NULL) {
+        if (!PyBytes_Check(result)) {
+            PyErr_SetString(PyExc_TypeError,
+                            "first argument must be an open file");
+            Py_DECREF(result);
+            Py_DECREF(rargs);
+            Py_DECREF(reader);
+            return NULL;
+        }
+
+        nread = PyBytes_Size(result);
+
+        t = self->nbits;
+        p = setunused(self);
+        self->nbits += p;
+
+        newsize = Py_SIZE(self) + nread;
+
+        if (resize(self, BITS(newsize)) < 0) {
+            Py_DECREF(result);
+            Py_DECREF(rargs);
+            Py_DECREF(reader);
+            return NULL;
+        }
+
+        memcpy(self->ob_item + (Py_SIZE(self) - nread),
+               PyBytes_AS_STRING(result), nread);
+
+        if (nbytes > 0 && nread < (size_t) nbytes) {
+            PyErr_SetString(PyExc_EOFError, "not enough items read");
+            return NULL;
+        }
+        if (delete_n(self, t, p) < 0)
+            return NULL;
+        Py_DECREF(result);
+    }
+
+    Py_DECREF(rargs);
+    Py_DECREF(reader);
+
+    Py_RETURN_NONE;
+}
+#else
+static PyObject *
+bitarray_fromfile(bitarrayobject *self, PyObject *args)
+{
+    PyObject *f;
+    FILE *fp;
+    Py_ssize_t newsize, nbytes = -1;
+    size_t nread;
+    idx_t t, p;
+    long cur;
+
+    if (!PyArg_ParseTuple(args, "O|n:fromfile", &f, &nbytes))
+        return NULL;
+
+    fp = PyFile_AsFile(f);
+    if (fp == NULL) {
+        PyErr_SetString(PyExc_TypeError,
+                        "first argument must be an open file");
+        return NULL;
+    }
+
+    /* find number of bytes till EOF */
+    if (nbytes < 0) {
+        if ((cur = ftell(fp)) < 0)
+            goto EOFerror;
+
+        if (fseek(fp, 0L, SEEK_END) || (nbytes = ftell(fp)) < 0)
+            goto EOFerror;
+
+        nbytes -= cur;
+        if (fseek(fp, cur, SEEK_SET)) {
+        EOFerror:
+            PyErr_SetString(PyExc_EOFError, "could not find EOF");
+            return NULL;
+        }
+    }
+    if (nbytes == 0)
+        Py_RETURN_NONE;
+
+    /* file exists and there are more than zero bytes to read */
+    t = self->nbits;
+    p = setunused(self);
+    self->nbits += p;
+
+    newsize = Py_SIZE(self) + nbytes;
+    if (resize(self, BITS(newsize)) < 0)
+        return NULL;
+
+    nread = fread(self->ob_item + (Py_SIZE(self) - nbytes), 1, nbytes, fp);
+    if (nread < (size_t) nbytes) {
+        newsize -= nbytes - nread;
+        if (resize(self, BITS(newsize)) < 0)
+            return NULL;
+        PyErr_SetString(PyExc_EOFError, "not enough items in file");
+        return NULL;
+    }
+
+    if (delete_n(self, t, p) < 0)
+        return NULL;
+    Py_RETURN_NONE;
+}
+#endif
+
+PyDoc_STRVAR(fromfile_doc,
+"fromfile(f, [n])\n\
+\n\
+Read n bytes from the file object f and append them to the bitarray\n\
+interpreted as machine values.  When n is omitted, as many bytes are\n\
+read until EOF is reached.");
+
+
+#ifdef IS_PY3K
+static PyObject *
+bitarray_tofile(bitarrayobject *self, PyObject *f)
+{
+    PyObject *writer, *value, *args, *result;
+
+    if (f == NULL) {
+        PyErr_SetString(PyExc_TypeError, "writeobject with NULL file");
+        return NULL;
+    }
+    writer = PyObject_GetAttrString(f, "write");
+    if (writer == NULL)
+        return NULL;
+    setunused(self);
+    value = PyBytes_FromStringAndSize(self->ob_item, Py_SIZE(self));
+    if (value == NULL) {
+        Py_DECREF(writer);
+        return NULL;
+    }
+    args = PyTuple_Pack(1, value);
+    if (args == NULL) {
+        Py_DECREF(value);
+        Py_DECREF(writer);
+        return NULL;
+    }
+    result = PyEval_CallObject(writer, args);
+    Py_DECREF(args);
+    Py_DECREF(value);
+    Py_DECREF(writer);
+    if (result == NULL)
+    {
+        PyErr_SetString(PyExc_TypeError, "open file expected");
+        return NULL;
+    }
+    Py_DECREF(result);
+    Py_RETURN_NONE;
+}
+#else
+static PyObject *
+bitarray_tofile(bitarrayobject *self, PyObject *f)
+{
+    FILE *fp;
+
+    fp = PyFile_AsFile(f);
+    if (fp == NULL) {
+        PyErr_SetString(PyExc_TypeError, "open file expected");
+        return NULL;
+    }
+    if (Py_SIZE(self) == 0)
+        Py_RETURN_NONE;
+
+    setunused(self);
+    if (fwrite(self->ob_item, 1, Py_SIZE(self), fp) !=
+        (size_t) Py_SIZE(self))
+    {
+        PyErr_SetFromErrno(PyExc_IOError);
+        clearerr(fp);
+        return NULL;
+    }
+    Py_RETURN_NONE;
+}
+#endif
+
+PyDoc_STRVAR(tofile_doc,
+"tofile(f)\n\
+\n\
+Write all bits (as machine values) to the file object f.\n\
+When the length of the bitarray is not a multiple of 8,\n\
+the remaining bits (1..7) are set to 0.");
+
+
+static PyObject *
+bitarray_tolist(bitarrayobject *self)
+{
+    PyObject *list;
+    idx_t i;
+
+    list = PyList_New((Py_ssize_t) self->nbits);
+    if (list == NULL)
+        return NULL;
+
+    for (i = 0; i < self->nbits; i++)
+        if (PyList_SetItem(list, (Py_ssize_t) i,
+                           PyBool_FromLong(GETBIT(self, i))) < 0)
+            return NULL;
+    return list;
+}
+
+PyDoc_STRVAR(tolist_doc,
+"tolist() -> list\n\
+\n\
+Return an ordinary list with the items in the bitarray.\n\
+Note that the list object being created will require 32 or 64 times more\n\
+memory than the bitarray object, which may cause a memory error if the\n\
+bitarray is very large.\n\
+Also note that to extend a bitarray with elements from a list,\n\
+use the extend method.");
+
+
+static PyObject *
+bitarray_frombytes(bitarrayobject *self, PyObject *string)
+{
+    idx_t t, p;
+
+    if (!PyString_Check(string)) {
+        PyErr_SetString(PyExc_TypeError, "byte string expected");
+        return NULL;
+    }
+    t = self->nbits;
+    p = setunused(self);
+    self->nbits += p;
+
+    if (extend_rawstring(self, string) < 0)
+        return NULL;
+    if (delete_n(self, t, p) < 0)
+        return NULL;
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(frombytes_doc,
+"frombytes(bytes)\n\
+\n\
+Append from a byte string, interpreted as machine values.");
+
+
+static PyObject *
+bitarray_tobytes(bitarrayobject *self)
+{
+    setunused(self);
+    return PyString_FromStringAndSize(self->ob_item, Py_SIZE(self));
+}
+
+PyDoc_STRVAR(tobytes_doc,
+"tobytes() -> bytes\n\
+\n\
+Return the byte representation of the bitarray.\n\
+When the length of the bitarray is not a multiple of 8, the few remaining\n\
+bits (1..7) are set to 0.");
+
+
+static PyObject *
+bitarray_to01(bitarrayobject *self)
+{
+#ifdef IS_PY3K
+    PyObject *string, *unpacked;
+
+    unpacked = unpack(self, '0', '1');
+    string = PyUnicode_FromEncodedObject(unpacked, NULL, NULL);
+    Py_DECREF(unpacked);
+    return string;
+#else
+    return unpack(self, '0', '1');
+#endif
+}
+
+PyDoc_STRVAR(to01_doc,
+"to01() -> string\n\
+\n\
+Return a string containing '0's and '1's, representing the bits in the\n\
+bitarray object.\n\
+Note: To extend a bitarray from a string containing '0's and '1's,\n\
+use the extend method.");
+
+
+static PyObject *
+bitarray_unpack(bitarrayobject *self, PyObject *args, PyObject *kwds)
+{
+    char zero = 0x00, one = 0xff;
+    static char* kwlist[] = {"zero", "one", NULL};
+
+    if (!PyArg_ParseTupleAndKeywords(args, kwds, "|cc:unpack", kwlist,
+                                     &zero, &one))
+        return NULL;
+
+    return unpack(self, zero, one);
+}
+
+PyDoc_STRVAR(unpack_doc,
+"unpack(zero=b'\\x00', one=b'\\xff') -> bytes\n\
+\n\
+Return a byte string containing one character for each bit in the bitarray,\n\
+using the specified mapping.\n\
+See also the pack method.");
+
+
+static PyObject *
+bitarray_pack(bitarrayobject *self, PyObject *string)
+{
+    if (!PyString_Check(string)) {
+        PyErr_SetString(PyExc_TypeError, "byte string expected");
+        return NULL;
+    }
+    if (extend_string(self, string, STR_RAW) < 0)
+        return NULL;
+
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(pack_doc,
+"pack(bytes)\n\
+\n\
+Extend the bitarray from a byte string, where each characters corresponds to\n\
+a single bit.  The character b'\\x00' maps to bit 0 and all other characters\n\
+map to bit 1.\n\
+This method, as well as the unpack method, are meant for efficient\n\
+transfer of data between bitarray objects to other python objects\n\
+(for example NumPy's ndarray object) which have a different view of memory.");
+
+
+static PyObject *
+bitarray_repr(bitarrayobject *self)
+{
+    PyObject *string;
+#ifdef IS_PY3K
+    PyObject *decoded;
+#endif
+
+    if (self->nbits == 0) {
+        string = PyString_FromString("bitarray()");
+        if (string == NULL)
+            return NULL;
+    }
+    else {
+        string = PyString_FromString("bitarray(\'");
+        if (string == NULL)
+            return NULL;
+        PyString_ConcatAndDel(&string, unpack(self, '0', '1'));
+        PyString_ConcatAndDel(&string, PyString_FromString("\')"));
+    }
+#ifdef IS_PY3K
+    decoded = PyUnicode_FromEncodedObject(string, NULL, NULL);
+    Py_DECREF(string);
+    string = decoded;
+#endif
+    return string;
+}
+
+
+static PyObject *
+bitarray_insert(bitarrayobject *self, PyObject *args)
+{
+    idx_t i;
+    PyObject *v;
+
+    if (!PyArg_ParseTuple(args, "LO:insert", &i, &v))
+        return NULL;
+
+    if (i < 0) {
+        i += self->nbits;
+        if (i < 0)
+            i = 0;
+    }
+    if (i > self->nbits)
+        i = self->nbits;
+
+    if (insert_n(self, i, 1) < 0)
+        return NULL;
+    if (set_item(self, i, v) < 0)
+        return NULL;
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(insert_doc,
+"insert(i, item)\n\
+\n\
+Insert bool(item) into the bitarray before position i.");
+
+
+static PyObject *
+bitarray_pop(bitarrayobject *self, PyObject *args)
+{
+    idx_t i = -1;
+    long vi;
+
+    if (!PyArg_ParseTuple(args, "|L:pop", &i))
+        return NULL;
+
+    if (self->nbits == 0) {
+        /* special case -- most common failure cause */
+        PyErr_SetString(PyExc_IndexError, "pop from empty bitarray");
+        return NULL;
+    }
+    if (i < 0)
+        i += self->nbits;
+
+    if (i < 0 || i >= self->nbits) {
+        PyErr_SetString(PyExc_IndexError, "pop index out of range");
+        return NULL;
+    }
+    vi = GETBIT(self, i);
+    if (delete_n(self, i, 1) < 0)
+        return NULL;
+    return PyBool_FromLong(vi);
+}
+
+PyDoc_STRVAR(pop_doc,
+"pop([i]) -> item\n\
+\n\
+Return the i-th (default last) element and delete it from the bitarray.\n\
+Raises IndexError if bitarray is empty or index is out of range.");
+
+
+static PyObject *
+bitarray_remove(bitarrayobject *self, PyObject *v)
+{
+    idx_t i;
+    long vi;
+
+    vi = PyObject_IsTrue(v);
+    if (vi < 0)
+        return NULL;
+
+    i = findfirst(self, vi, 0, -1);
+    if (i < 0) {
+        PyErr_SetString(PyExc_ValueError, "remove(x): x not in bitarray");
+        return NULL;
+    }
+    if (delete_n(self, i, 1) < 0)
+        return NULL;
+    Py_RETURN_NONE;
+}
+
+PyDoc_STRVAR(remove_doc,
+"remove(item)\n\
+\n\
+Remove the first occurrence of bool(item) in the bitarray.\n\
+Raises ValueError if item is not present.");
+
+
+/* --------- special methods ----------- */
+
+static PyObject *
+bitarray_getitem(bitarrayobject *self, PyObject *a)
+{
+    PyObject *res;
+    idx_t start, stop, step, slicelength, j, i = 0;
+
+    if (IS_INDEX(a)) {
+        if (getIndex(a, &i) < 0)
+            return NULL;
+        if (i < 0)
+            i += self->nbits;
+        if (i < 0 || i >= self->nbits) {
+            PyErr_SetString(PyExc_IndexError, "bitarray index out of range");
+            return NULL;
+        }
+        return PyBool_FromLong(GETBIT(self, i));
+    }
+    if (PySlice_Check(a)) {
+        if (slice_GetIndicesEx((PySliceObject *) a, self->nbits,
+                               &start, &stop, &step, &slicelength) < 0) {
+            return NULL;
+        }
+        res = newbitarrayobject(Py_TYPE(self), slicelength, self->endian);
+        if (res == NULL)
+            return NULL;
+
+        for (i = 0, j = start; i < slicelength; i++, j += step)
+            setbit((bitarrayobject *) res, i, GETBIT(self, j));
+
+        return res;
+    }
+    PyErr_SetString(PyExc_TypeError, "index or slice expected");
+    return NULL;
+}
+
+/* Sets the elements, specified by slice, in self to the value(s) given by v
+   which is either a bitarray or a boolean.
+*/
+static int
+setslice(bitarrayobject *self, PySliceObject *slice, PyObject *v)
+{
+    idx_t start, stop, step, slicelength, j, i = 0;
+
+    if (slice_GetIndicesEx(slice, self->nbits,
+                           &start, &stop, &step, &slicelength) < 0)
+        return -1;
+
+    if (bitarray_Check(v)) {
+#define vv  ((bitarrayobject *) v)
+        if (vv->nbits == slicelength) {
+            for (i = 0, j = start; i < slicelength; i++, j += step)
+                setbit(self, j, GETBIT(vv, i));
+            return 0;
+        }
+        if (step != 1) {
+            char buff[256];
+            sprintf(buff, "attempt to assign sequence of size %lld "
+                          "to extended slice of size %lld",
+                    vv->nbits, (idx_t) slicelength);
+            PyErr_SetString(PyExc_ValueError, buff);
+            return -1;
+        }
+        /* make self bigger or smaller */
+        if (vv->nbits > slicelength) {
+            if (insert_n(self, start, vv->nbits - slicelength) < 0)
+                return -1;
+        }
+        else {
+            if (delete_n(self, start, slicelength - vv->nbits) < 0)
+                return -1;
+        }
+        /* copy the new values into self */
+        copy_n(self, start, vv, 0, vv->nbits);
+#undef vv
+        return 0;
+    }
+    if (IS_INT_OR_BOOL(v)) {
+        int vi;
+
+        vi = IntBool_AsInt(v);
+        if (vi < 0)
+            return -1;
+        for (i = 0, j = start; i < slicelength; i++, j += step)
+            setbit(self, j, vi);
+        return 0;
+    }
+    PyErr_SetString(PyExc_IndexError,
+                    "bitarray or bool expected for slice assignment");
+    return -1;
+}
+
+static PyObject *
+bitarray_setitem(bitarrayobject *self, PyObject *args)
+{
+    PyObject *a, *v;
+    idx_t i = 0;
+
+    if (!PyArg_ParseTuple(args, "OO:__setitem__", &a, &v))
+        return NULL;
+
+    if (IS_INDEX(a)) {
+        if (getIndex(a, &i) < 0)
+            return NULL;
+        if (i < 0)
+            i += self->nbits;
+        if (i < 0 || i >= self->nbits) {
+            PyErr_SetString(PyExc_IndexError, "bitarray index out of range");
+            return NULL;
+        }
+        if (set_item(self, i, v) < 0)
+            return NULL;
+        Py_RETURN_NONE;
+    }
+    if (PySlice_Check(a)) {
+        if (setslice(self, (PySliceObject *) a, v) < 0)
+            return NULL;
+        Py_RETURN_NONE;
+    }
+    PyErr_SetString(PyExc_TypeError, "index or slice expected");
+    return NULL;
+}
+
+static PyObject *
+bitarray_delitem(bitarrayobject *self, PyObject *a)
+{
+    idx_t start, stop, step, slicelength, j, i = 0;
+
+    if (IS_INDEX(a)) {
+        if (getIndex(a, &i) < 0)
+            return NULL;
+        if (i < 0)
+            i += self->nbits;
+        if (i < 0 || i >= self->nbits) {
+            PyErr_SetString(PyExc_IndexError, "bitarray index out of range");
+            return NULL;
+        }
+        if (delete_n(self, i, 1) < 0)
+            return NULL;
+        Py_RETURN_NONE;
+    }
+    if (PySlice_Check(a)) {
+        if (slice_GetIndicesEx((PySliceObject *) a, self->nbits,
+                               &start, &stop, &step, &slicelength) < 0) {
+            return NULL;
+        }
+        if (slicelength == 0)
+            Py_RETURN_NONE;
+
+        if (step < 0) {
+            stop = start + 1;
+            start = stop + step * (slicelength - 1) - 1;
+            step = -step;
+        }
+        if (step == 1) {
+            assert(stop - start == slicelength);
+            if (delete_n(self, start, slicelength) < 0)
+                return NULL;
+            Py_RETURN_NONE;
+        }
+        /* this is the only complicated part when step > 1 */
+        for (i = j = start; i < self->nbits; i++)
+            if ((i - start) % step != 0 || i >= stop) {
+                setbit(self, j, GETBIT(self, i));
+                j++;
+            }
+        if (resize(self, self->nbits - slicelength) < 0)
+            return NULL;
+        Py_RETURN_NONE;
+    }
+    PyErr_SetString(PyExc_TypeError, "index or slice expected");
+    return NULL;
+}
+
+/* ---------- number methods ---------- */
+
+static PyObject *
+bitarray_add(bitarrayobject *self, PyObject *other)
+{
+    PyObject *res;
+
+    res = bitarray_copy(self);
+    if (extend_dispatch((bitarrayobject *) res, other) < 0) {
+        Py_DECREF(res);
+        return NULL;
+    }
+    return res;
+}
+
+static PyObject *
+bitarray_iadd(bitarrayobject *self, PyObject *other)
+{
+    if (extend_dispatch(self, other) < 0)
+        return NULL;
+    Py_INCREF(self);
+    return (PyObject *) self;
+}
+
+static PyObject *
+bitarray_mul(bitarrayobject *self, PyObject *v)
+{
+    PyObject *res;
+    idx_t vi = 0;
+
+    if (!IS_INDEX(v)) {
+        PyErr_SetString(PyExc_TypeError,
+                        "integer value expected for bitarray repetition");
+        return NULL;
+    }
+    if (getIndex(v, &vi) < 0)
+        return NULL;
+    res = bitarray_copy(self);
+    if (repeat((bitarrayobject *) res, vi) < 0) {
+        Py_DECREF(res);
+        return NULL;
+    }
+    return res;
+}
+
+static PyObject *
+bitarray_imul(bitarrayobject *self, PyObject *v)
+{
+    idx_t vi = 0;
+
+    if (!IS_INDEX(v)) {
+        PyErr_SetString(PyExc_TypeError,
+            "integer value expected for in-place bitarray repetition");
+        return NULL;
+    }
+    if (getIndex(v, &vi) < 0)
+        return NULL;
+    if (repeat(self, vi) < 0)
+        return NULL;
+    Py_INCREF(self);
+    return (PyObject *) self;
+}
+
+static PyObject *
+bitarray_cpinvert(bitarrayobject *self)
+{
+    PyObject *res;
+
+    res = bitarray_copy(self);
+    invert((bitarrayobject *) res);
+    return res;
+}
+
+#define BITWISE_FUNC(oper)  \
+static PyObject *                                                   \
+bitarray_ ## oper (bitarrayobject *self, PyObject *other)           \
+{                                                                   \
+    PyObject *res;                                                  \
+                                                                    \
+    res = bitarray_copy(self);                                      \
+    if (bitwise((bitarrayobject *) res, other, OP_ ## oper) < 0) {  \
+        Py_DECREF(res);                                             \
+        return NULL;                                                \
+    }                                                               \
+    return res;                                                     \
+}
+
+BITWISE_FUNC(and)
+BITWISE_FUNC(or)
+BITWISE_FUNC(xor)
+
+
+#define BITWISE_IFUNC(oper)  \
+static PyObject *                                            \
+bitarray_i ## oper (bitarrayobject *self, PyObject *other)   \
+{                                                            \
+    if (bitwise(self, other, OP_ ## oper) < 0)               \
+        return NULL;                                         \
+    Py_INCREF(self);                                         \
+    return (PyObject *) self;                                \
+}
+
+BITWISE_IFUNC(and)
+BITWISE_IFUNC(or)
+BITWISE_IFUNC(xor)
+
+/******************* variable length encoding and decoding ***************/
+
+static PyObject *
+bitarray_encode(bitarrayobject *self, PyObject *args)
+{
+    PyObject *codedict, *iterable, *iter, *symbol, *bits;
+
+    if (!PyArg_ParseTuple(args, "OO:_encode", &codedict, &iterable))
+        return NULL;
+
+    iter = PyObject_GetIter(iterable);
+    if (iter == NULL) {
+        PyErr_SetString(PyExc_TypeError, "iterable object expected");
+        return NULL;
+    }
+    /* extend self with the bitarrays from codedict */
+    while ((symbol = PyIter_Next(iter)) != NULL) {
+        bits = PyDict_GetItem(codedict, symbol);
+        Py_DECREF(symbol);
+        if (bits == NULL) {
+            PyErr_SetString(PyExc_ValueError, "symbol not in prefix code");
+            goto error;
+        }
+        if (extend_bitarray(self, (bitarrayobject *) bits) < 0)
+            goto error;
+    }
+    Py_DECREF(iter);
+    if (PyErr_Occurred())
+        return NULL;
+    Py_RETURN_NONE;
+error:
+    Py_DECREF(iter);
+    return NULL;
+}
+
+PyDoc_STRVAR(encode_doc,
+"_encode(code, iterable)\n\
+\n\
+like the encode method without code checking");
+
+
+/* Binary Tree definition */
+typedef struct _bin_node
+{
+    PyObject *symbol;
+    struct _bin_node *child[2];
+} binode;
+
+
+static binode *
+new_binode(void)
+{
+    binode *nd;
+
+    nd = PyMem_Malloc(sizeof *nd);
+    if (nd == NULL) {
+        PyErr_NoMemory();
+        return NULL;
+    }
+    nd->symbol = NULL;
+    nd->child[0] = NULL;
+    nd->child[1] = NULL;
+    return nd;
+}
+
+static void
+delete_binode_tree(binode *root)
+{
+    if (root == NULL)
+        return;
+
+    delete_binode_tree(root->child[0]);
+    delete_binode_tree(root->child[1]);
+    PyMem_Free(root);
+}
+
+static int
+insert_symbol(binode *root, bitarrayobject *self, PyObject *symbol)
+{
+    binode *nd = root, *prev;
+    Py_ssize_t i;
+    int k;
+
+    for (i = 0; i < self->nbits; i++) {
+        k = GETBIT(self, i);
+        prev = nd;
+        nd = nd->child[k];
+        if (!nd) {
+            nd = new_binode();
+            if (nd == NULL)
+                return -1;
+            prev->child[k] = nd;
+        }
+    }
+
+    if (nd->symbol) {
+        PyErr_SetString(PyExc_ValueError, "prefix code ambiguous");
+        return -1;
+    }
+    nd->symbol = symbol;
+    return 0;
+}
+
+static binode *
+make_tree(PyObject *codedict)
+{
+    binode *root;
+    PyObject *symbol, *array;
+    Py_ssize_t pos = 0;
+
+    root = new_binode();
+    if (root == NULL)
+        return NULL;
+
+    while (PyDict_Next(codedict, &pos, &symbol, &array)) {
+        if (insert_symbol(root, (bitarrayobject *) array, symbol) < 0) {
+            delete_binode_tree(root);
+            return NULL;
+        }
+    }
+    return root;
+}
+
+static PyObject *
+tree_traverse(bitarrayobject *self, idx_t *indexp, binode *tree)
+{
+    binode *nd = tree;
+    int k;
+
+    while (*indexp < self->nbits) {
+        k = GETBIT(self, *indexp);
+        (*indexp)++;
+        nd = nd->child[k];
+        if (nd == NULL) {
+            PyErr_SetString(PyExc_ValueError,
+                            "prefix code does not match data in bitarray");
+            return NULL;
+        }
+        if (nd->symbol)  /* leaf */
+            return nd->symbol;
+    }
+    if (nd != tree)
+        PyErr_SetString(PyExc_ValueError, "decoding not terminated");
+
+    return NULL;
+}
+
+static PyObject *
+bitarray_decode(bitarrayobject *self, PyObject *codedict)
+{
+    binode *tree, *nd;
+    PyObject *list;
+    Py_ssize_t i;
+    int k;
+
+    tree = make_tree(codedict);
+    if (tree == NULL || PyErr_Occurred())
+        return NULL;
+
+    nd = tree;
+    list = PyList_New(0);
+    if (list == NULL) {
+        delete_binode_tree(tree);
+        return NULL;
+    }
+    for (i = 0; i < self->nbits; i++) {
+        k = GETBIT(self, i);
+        nd = nd->child[k];
+        if (nd == NULL) {
+            PyErr_SetString(PyExc_ValueError,
+                            "prefix code does not match data in bitarray");
+            goto error;
+        }
+        if (nd->symbol) {  /* leaf */
+            if (PyList_Append(list, nd->symbol) < 0)
+                goto error;
+            nd = tree;
+        }
+    }
+    if (nd != tree) {
+        PyErr_SetString(PyExc_ValueError, "decoding not terminated");
+        goto error;
+    }
+    delete_binode_tree(tree);
+    return list;
+
+error:
+    delete_binode_tree(tree);
+    Py_DECREF(list);
+    return NULL;
+}
+
+PyDoc_STRVAR(decode_doc,
+"_decode(codedict) -> list\n\
+\n\
+Given a code dictionary, decode the content of the bitarray and return\n\
+the list of symbols.");
+
+/*********************** (Bitarray) Decode Iterator *********************/
+
+
+typedef struct {
+    PyObject_HEAD
+    bitarrayobject *bao;        /* bitarray we're searching in */
+    binode *tree;               /* prefix tree containing symbols */
+    idx_t index;                /* current index in bitarray */
+} decodeiterobject;
+
+static PyTypeObject DecodeIter_Type;
+
+#define DecodeIter_Check(op)  PyObject_TypeCheck(op, &DecodeIter_Type)
+
+
+
+/* create a new initialized bitarray search iterator object */
+static PyObject *
+bitarray_iterdecode(bitarrayobject *self, PyObject *codedict)
+{
+    decodeiterobject *it;  /* iterator to be returned */
+    binode *tree;
+
+    tree = make_tree(codedict);
+    if (tree == NULL || PyErr_Occurred())
+        return NULL;
+
+    it = PyObject_GC_New(decodeiterobject, &DecodeIter_Type);
+    if (it == NULL)
+        return NULL;
+
+    it->tree = tree;
+
+    Py_INCREF(self);
+    it->bao = self;
+    it->index = 0;
+    PyObject_GC_Track(it);
+    return (PyObject *) it;
+}
+
+PyDoc_STRVAR(iterdecode_doc,
+"_iterdecode(codedict) -> iterator\n\
+\n\
+Given a code dictionary, decode the content of the bitarray and iterate\n\
+over the represented symbols.");
+
+static PyObject *
+decodeiter_next(decodeiterobject *it)
+{
+    PyObject *symbol;
+
+    assert(DecodeIter_Check(it));
+    symbol = tree_traverse(it->bao, &(it->index), it->tree);
+    if (symbol == NULL)  /* stop iteration OR error occured */
+        return NULL;
+    Py_INCREF(symbol);
+    return symbol;
+}
+
+static void
+decodeiter_dealloc(decodeiterobject *it)
+{
+    delete_binode_tree(it->tree);
+    PyObject_GC_UnTrack(it);
+    Py_XDECREF(it->bao);
+    PyObject_GC_Del(it);
+}
+
+static int
+decodeiter_traverse(decodeiterobject *it, visitproc visit, void *arg)
+{
+    Py_VISIT(it->bao);
+    return 0;
+}
+
+static PyTypeObject DecodeIter_Type = {
+#ifdef IS_PY3K
+    PyVarObject_HEAD_INIT(&DecodeIter_Type, 0)
+#else
+    PyObject_HEAD_INIT(NULL)
+    0,                                        /* ob_size */
+#endif
+    "bitarraydecodeiterator",                 /* tp_name */
+    sizeof(decodeiterobject),                 /* tp_basicsize */
+    0,                                        /* tp_itemsize */
+    /* methods */
+    (destructor) decodeiter_dealloc,          /* tp_dealloc */
+    0,                                        /* tp_print */
+    0,                                        /* tp_getattr */
+    0,                                        /* tp_setattr */
+    0,                                        /* tp_compare */
+    0,                                        /* tp_repr */
+    0,                                        /* tp_as_number */
+    0,                                        /* tp_as_sequence */
+    0,                                        /* tp_as_mapping */
+    0,                                        /* tp_hash */
+    0,                                        /* tp_call */
+    0,                                        /* tp_str */
+    PyObject_GenericGetAttr,                  /* tp_getattro */
+    0,                                        /* tp_setattro */
+    0,                                        /* tp_as_buffer */
+    Py_TPFLAGS_DEFAULT | Py_TPFLAGS_HAVE_GC,  /* tp_flags */
+    0,                                        /* tp_doc */
+    (traverseproc) decodeiter_traverse,       /* tp_traverse */
+    0,                                        /* tp_clear */
+    0,                                        /* tp_richcompare */
+    0,                                        /* tp_weaklistoffset */
+    PyObject_SelfIter,                        /* tp_iter */
+    (iternextfunc) decodeiter_next,           /* tp_iternext */
+    0,                                        /* tp_methods */
+};
+
+/*********************** (Bitarray) Search Iterator *********************/
+
+typedef struct {
+    PyObject_HEAD
+    bitarrayobject *bao;        /* bitarray we're searching in */
+    bitarrayobject *xa;         /* bitarray being searched for */
+    idx_t p;                    /* current search position */
+} searchiterobject;
+
+static PyTypeObject SearchIter_Type;
+
+#define SearchIter_Check(op)  PyObject_TypeCheck(op, &SearchIter_Type)
+
+/* create a new initialized bitarray search iterator object */
+static PyObject *
+bitarray_itersearch(bitarrayobject *self, PyObject *x)
+{
+    searchiterobject *it;  /* iterator to be returned */
+    bitarrayobject *xa;
+
+    if (!bitarray_Check(x)) {
+        PyErr_SetString(PyExc_TypeError, "bitarray expected for itersearch");
+        return NULL;
+    }
+    xa = (bitarrayobject *) x;
+    if (xa->nbits == 0) {
+        PyErr_SetString(PyExc_ValueError, "can't search for empty bitarray");
+        return NULL;
+    }
+
+    it = PyObject_GC_New(searchiterobject, &SearchIter_Type);
+    if (it == NULL)
+        return NULL;
+
+    Py_INCREF(self);
+    it->bao = self;
+    Py_INCREF(xa);
+    it->xa = xa;
+    it->p = 0;  /* start search at position 0 */
+    PyObject_GC_Track(it);
+    return (PyObject *) it;
+}
+
+PyDoc_STRVAR(itersearch_doc,
+"itersearch(bitarray) -> iterator\n\
+\n\
+Searches for the given a bitarray in self, and return an iterator over\n\
+the start positions where bitarray matches self.");
+
+static PyObject *
+searchiter_next(searchiterobject *it)
+{
+    idx_t p;
+
+    assert(SearchIter_Check(it));
+    p = search(it->bao, it->xa, it->p);
+    if (p < 0)  /* no more positions -- stop iteration */
+        return NULL;
+    it->p = p + 1;  /* next search position */
+    return PyLong_FromLongLong(p);
+}
+
+static void
+searchiter_dealloc(searchiterobject *it)
+{
+    PyObject_GC_UnTrack(it);
+    Py_XDECREF(it->bao);
+    Py_XDECREF(it->xa);
+    PyObject_GC_Del(it);
+}
+
+static int
+searchiter_traverse(searchiterobject *it, visitproc visit, void *arg)
+{
+    Py_VISIT(it->bao);
+    return 0;
+}
+
+static PyTypeObject SearchIter_Type = {
+#ifdef IS_PY3K
+    PyVarObject_HEAD_INIT(&SearchIter_Type, 0)
+#else
+    PyObject_HEAD_INIT(NULL)
+    0,                                        /* ob_size */
+#endif
+    "bitarraysearchiterator",                 /* tp_name */
+    sizeof(searchiterobject),                 /* tp_basicsize */
+    0,                                        /* tp_itemsize */
+    /* methods */
+    (destructor) searchiter_dealloc,          /* tp_dealloc */
+    0,                                        /* tp_print */
+    0,                                        /* tp_getattr */
+    0,                                        /* tp_setattr */
+    0,                                        /* tp_compare */
+    0,                                        /* tp_repr */
+    0,                                        /* tp_as_number */
+    0,                                        /* tp_as_sequence */
+    0,                                        /* tp_as_mapping */
+    0,                                        /* tp_hash */
+    0,                                        /* tp_call */
+    0,                                        /* tp_str */
+    PyObject_GenericGetAttr,                  /* tp_getattro */
+    0,                                        /* tp_setattro */
+    0,                                        /* tp_as_buffer */
+    Py_TPFLAGS_DEFAULT | Py_TPFLAGS_HAVE_GC,  /* tp_flags */
+    0,                                        /* tp_doc */
+    (traverseproc) searchiter_traverse,       /* tp_traverse */
+    0,                                        /* tp_clear */
+    0,                                        /* tp_richcompare */
+    0,                                        /* tp_weaklistoffset */
+    PyObject_SelfIter,                        /* tp_iter */
+    (iternextfunc) searchiter_next,           /* tp_iternext */
+    0,                                        /* tp_methods */
+};
+
+/*************************** Method definitions *************************/
+
+static PyMethodDef
+bitarray_methods[] = {
+    {"all",          (PyCFunction) bitarray_all,         METH_NOARGS,
+     all_doc},
+    {"any",          (PyCFunction) bitarray_any,         METH_NOARGS,
+     any_doc},
+    {"append",       (PyCFunction) bitarray_append,      METH_O,
+     append_doc},
+    {"buffer_info",  (PyCFunction) bitarray_buffer_info, METH_NOARGS,
+     buffer_info_doc},
+    {"bytereverse",  (PyCFunction) bitarray_bytereverse, METH_NOARGS,
+     bytereverse_doc},
+    {"copy",         (PyCFunction) bitarray_copy,        METH_NOARGS,
+     copy_doc},
+    {"count",        (PyCFunction) bitarray_count,       METH_VARARGS,
+     count_doc},
+    {"_decode",      (PyCFunction) bitarray_decode,      METH_O,
+     decode_doc},
+    {"_iterdecode",  (PyCFunction) bitarray_iterdecode,  METH_O,
+     iterdecode_doc},
+    {"_encode",      (PyCFunction) bitarray_encode,      METH_VARARGS,
+     encode_doc},
+    {"endian",       (PyCFunction) bitarray_endian,      METH_NOARGS,
+     endian_doc},
+    {"extend",       (PyCFunction) bitarray_extend,      METH_O,
+     extend_doc},
+    {"fill",         (PyCFunction) bitarray_fill,        METH_NOARGS,
+     fill_doc},
+    {"fromfile",     (PyCFunction) bitarray_fromfile,    METH_VARARGS,
+     fromfile_doc},
+    {"frombytes",    (PyCFunction) bitarray_frombytes,   METH_O,
+     frombytes_doc},
+    {"index",        (PyCFunction) bitarray_index,       METH_VARARGS,
+     index_doc},
+    {"insert",       (PyCFunction) bitarray_insert,      METH_VARARGS,
+     insert_doc},
+    {"invert",       (PyCFunction) bitarray_invert,      METH_NOARGS,
+     invert_doc},
+    {"length",       (PyCFunction) bitarray_length,      METH_NOARGS,
+     length_doc},
+    {"pack",         (PyCFunction) bitarray_pack,        METH_O,
+     pack_doc},
+    {"pop",          (PyCFunction) bitarray_pop,         METH_VARARGS,
+     pop_doc},
+    {"remove",       (PyCFunction) bitarray_remove,      METH_O,
+     remove_doc},
+    {"reverse",      (PyCFunction) bitarray_reverse,     METH_NOARGS,
+     reverse_doc},
+    {"setall",       (PyCFunction) bitarray_setall,      METH_O,
+     setall_doc},
+    {"search",       (PyCFunction) bitarray_search,      METH_VARARGS,
+     search_doc},
+    {"itersearch",   (PyCFunction) bitarray_itersearch,  METH_O,
+     itersearch_doc},
+    {"sort",         (PyCFunction) bitarray_sort,        METH_VARARGS |
+                                                         METH_KEYWORDS,
+     sort_doc},
+    {"tofile",       (PyCFunction) bitarray_tofile,      METH_O,
+     tofile_doc},
+    {"tolist",       (PyCFunction) bitarray_tolist,      METH_NOARGS,
+     tolist_doc},
+    {"tobytes",      (PyCFunction) bitarray_tobytes,     METH_NOARGS,
+     tobytes_doc},
+    {"to01",         (PyCFunction) bitarray_to01,        METH_NOARGS,
+     to01_doc},
+    {"unpack",       (PyCFunction) bitarray_unpack,      METH_VARARGS |
+                                                         METH_KEYWORDS,
+     unpack_doc},
+
+    /* special methods */
+    {"__copy__",     (PyCFunction) bitarray_copy,        METH_NOARGS,
+     copy_doc},
+    {"__deepcopy__", (PyCFunction) bitarray_copy,        METH_O,
+     copy_doc},
+    {"__len__",      (PyCFunction) bitarray_length,      METH_NOARGS,
+     len_doc},
+    {"__contains__", (PyCFunction) bitarray_contains,    METH_O,
+     contains_doc},
+    {"__reduce__",   (PyCFunction) bitarray_reduce,      METH_NOARGS,
+     reduce_doc},
+
+    /* slice methods */
+    {"__delitem__",  (PyCFunction) bitarray_delitem,     METH_O,       0},
+    {"__getitem__",  (PyCFunction) bitarray_getitem,     METH_O,       0},
+    {"__setitem__",  (PyCFunction) bitarray_setitem,     METH_VARARGS, 0},
+
+    /* number methods */
+    {"__add__",      (PyCFunction) bitarray_add,         METH_O,       0},
+    {"__iadd__",     (PyCFunction) bitarray_iadd,        METH_O,       0},
+    {"__mul__",      (PyCFunction) bitarray_mul,         METH_O,       0},
+    {"__rmul__",     (PyCFunction) bitarray_mul,         METH_O,       0},
+    {"__imul__",     (PyCFunction) bitarray_imul,        METH_O,       0},
+    {"__and__",      (PyCFunction) bitarray_and,         METH_O,       0},
+    {"__or__",       (PyCFunction) bitarray_or,          METH_O,       0},
+    {"__xor__",      (PyCFunction) bitarray_xor,         METH_O,       0},
+    {"__iand__",     (PyCFunction) bitarray_iand,        METH_O,       0},
+    {"__ior__",      (PyCFunction) bitarray_ior,         METH_O,       0},
+    {"__ixor__",     (PyCFunction) bitarray_ixor,        METH_O,       0},
+    {"__invert__",   (PyCFunction) bitarray_cpinvert,    METH_NOARGS,  0},
+
+    {NULL,           NULL}  /* sentinel */
+};
+
+
+static PyObject *
+bitarray_new(PyTypeObject *type, PyObject *args, PyObject *kwds)
+{
+    PyObject *a;  /* to be returned in some cases */
+    PyObject *initial = NULL;
+    char *endian_str = NULL;
+    int endian;
+    static char* kwlist[] = {"initial", "endian", NULL};
+
+    if (!PyArg_ParseTupleAndKeywords(args, kwds,
+                        "|Os:bitarray", kwlist, &initial, &endian_str))
+        return NULL;
+
+    if (endian_str == NULL) {
+        endian = DEFAULT_ENDIAN;  /* use default value */
+    }
+    else if (strcmp(endian_str, "little") == 0) {
+        endian = 0;
+    }
+    else if (strcmp(endian_str, "big") == 0) {
+        endian = 1;
+    }
+    else {
+        PyErr_SetString(PyExc_ValueError,
+                        "endian must be 'little' or 'big'");
+        return NULL;
+    }
+
+    /* no arg or None */
+    if (initial == NULL || initial == Py_None)
+        return newbitarrayobject(type, 0, endian);
+
+    /* int, long */
+    if (IS_INDEX(initial)) {
+        idx_t nbits = 0;
+
+        if (getIndex(initial, &nbits) < 0)
+            return NULL;
+        if (nbits < 0) {
+            PyErr_SetString(PyExc_ValueError,
+                            "cannot create bitarray with negative length");
+            return NULL;
+        }
+        return newbitarrayobject(type, nbits, endian);
+    }
+
+    /* from bitarray itself */
+    if (bitarray_Check(initial)) {
+#define np  ((bitarrayobject *) initial)
+        a = newbitarrayobject(type, np->nbits,
+                              endian_str == NULL ? np->endian : endian);
+        if (a == NULL)
+            return NULL;
+        memcpy(((bitarrayobject *) a)->ob_item, np->ob_item, Py_SIZE(np));
+#undef np
+        return a;
+    }
+
+    /* string */
+    if (PyString_Check(initial)) {
+        Py_ssize_t strlen;
+        char *str;
+
+        strlen = PyString_Size(initial);
+        if (strlen == 0)        /* empty string */
+            return newbitarrayobject(type, 0, endian);
+
+        str = PyString_AsString(initial);
+        if (0 <= str[0] && str[0] < 8) {
+            /* when the first character is smaller than 8, it indicates the
+               number of unused bits at the end, and rest of the bytes
+               consist of the raw binary data, this is used for pickling */
+            if (strlen == 1 && str[0] > 0) {
+                PyErr_Format(PyExc_ValueError,
+                             "did not expect 0x0%d", (int) str[0]);
+                return NULL;
+            }
+            a = newbitarrayobject(type, BITS(strlen - 1) - ((idx_t) str[0]),
+                                  endian);
+            if (a == NULL)
+                return NULL;
+            memcpy(((bitarrayobject *) a)->ob_item, str + 1, strlen - 1);
+            return a;
+        }
+    }
+
+    /* leave remaining type dispatch to the extend method */
+    a = newbitarrayobject(type, 0, endian);
+    if (a == NULL)
+        return NULL;
+    if (extend_dispatch((bitarrayobject *) a, initial) < 0) {
+        Py_DECREF(a);
+        return NULL;
+    }
+    return a;
+}
+
+
+static PyObject *
+richcompare(PyObject *v, PyObject *w, int op)
+{
+    int cmp, vi, wi;
+    idx_t i, vs, ws;
+
+    if (!bitarray_Check(v) || !bitarray_Check(w)) {
+        Py_INCREF(Py_NotImplemented);
+        return Py_NotImplemented;
+    }
+#define va  ((bitarrayobject *) v)
+#define wa  ((bitarrayobject *) w)
+    vs = va->nbits;
+    ws = wa->nbits;
+    if (vs != ws) {
+        /* shortcut for EQ/NE: if sizes differ, the bitarrays differ */
+        if (op == Py_EQ)
+            Py_RETURN_FALSE;
+        if (op == Py_NE)
+            Py_RETURN_TRUE;
+    }
+
+    /* to avoid uninitialized warning for some compilers */
+    vi = wi = 0;
+    /* search for the first index where items are different */
+    for (i = 0; i < vs && i < ws; i++) {
+        vi = GETBIT(va, i);
+        wi = GETBIT(wa, i);
+        if (vi != wi) {
+            /* we have an item that differs -- first, shortcut for EQ/NE */
+            if (op == Py_EQ)
+                Py_RETURN_FALSE;
+            if (op == Py_NE)
+                Py_RETURN_TRUE;
+            /* compare the final item using the proper operator */
+            switch (op) {
+            case Py_LT: cmp = vi <  wi; break;
+            case Py_LE: cmp = vi <= wi; break;
+            case Py_EQ: cmp = vi == wi; break;
+            case Py_NE: cmp = vi != wi; break;
+            case Py_GT: cmp = vi >  wi; break;
+            case Py_GE: cmp = vi >= wi; break;
+            default: return NULL;  /* cannot happen */
+            }
+            return PyBool_FromLong((long) cmp);
+        }
+    }
+#undef va
+#undef wa
+
+    /* no more items to compare -- compare sizes */
+    switch (op) {
+    case Py_LT: cmp = vs <  ws; break;
+    case Py_LE: cmp = vs <= ws; break;
+    case Py_EQ: cmp = vs == ws; break;
+    case Py_NE: cmp = vs != ws; break;
+    case Py_GT: cmp = vs >  ws; break;
+    case Py_GE: cmp = vs >= ws; break;
+    default: return NULL;  /* cannot happen */
+    }
+    return PyBool_FromLong((long) cmp);
+}
+
+/************************** Bitarray Iterator **************************/
+
+typedef struct {
+    PyObject_HEAD
+    bitarrayobject *bao;        /* bitarray we're iterating over */
+    idx_t index;                /* current index in bitarray */
+} bitarrayiterobject;
+
+static PyTypeObject BitarrayIter_Type;
+
+#define BitarrayIter_Check(op)  PyObject_TypeCheck(op, &BitarrayIter_Type)
+
+/* create a new initialized bitarray iterator object, this object is
+   returned when calling item(a) */
+static PyObject *
+bitarray_iter(bitarrayobject *self)
+{
+    bitarrayiterobject *it;
+
+    assert(bitarray_Check(self));
+    it = PyObject_GC_New(bitarrayiterobject, &BitarrayIter_Type);
+    if (it == NULL)
+        return NULL;
+
+    Py_INCREF(self);
+    it->bao = self;
+    it->index = 0;
+    PyObject_GC_Track(it);
+    return (PyObject *) it;
+}
+
+static PyObject *
+bitarrayiter_next(bitarrayiterobject *it)
+{
+    long vi;
+
+    assert(BitarrayIter_Check(it));
+    if (it->index < it->bao->nbits) {
+        vi = GETBIT(it->bao, it->index);
+        it->index++;
+        return PyBool_FromLong(vi);
+    }
+    return NULL;  /* stop iteration */
+}
+
+static void
+bitarrayiter_dealloc(bitarrayiterobject *it)
+{
+    PyObject_GC_UnTrack(it);
+    Py_XDECREF(it->bao);
+    PyObject_GC_Del(it);
+}
+
+static int
+bitarrayiter_traverse(bitarrayiterobject *it, visitproc visit, void *arg)
+{
+    Py_VISIT(it->bao);
+    return 0;
+}
+
+static PyTypeObject BitarrayIter_Type = {
+#ifdef IS_PY3K
+    PyVarObject_HEAD_INIT(&BitarrayIter_Type, 0)
+#else
+    PyObject_HEAD_INIT(NULL)
+    0,                                        /* ob_size */
+#endif
+    "bitarrayiterator",                       /* tp_name */
+    sizeof(bitarrayiterobject),               /* tp_basicsize */
+    0,                                        /* tp_itemsize */
+    /* methods */
+    (destructor) bitarrayiter_dealloc,        /* tp_dealloc */
+    0,                                        /* tp_print */
+    0,                                        /* tp_getattr */
+    0,                                        /* tp_setattr */
+    0,                                        /* tp_compare */
+    0,                                        /* tp_repr */
+    0,                                        /* tp_as_number */
+    0,                                        /* tp_as_sequence */
+    0,                                        /* tp_as_mapping */
+    0,                                        /* tp_hash */
+    0,                                        /* tp_call */
+    0,                                        /* tp_str */
+    PyObject_GenericGetAttr,                  /* tp_getattro */
+    0,                                        /* tp_setattro */
+    0,                                        /* tp_as_buffer */
+    Py_TPFLAGS_DEFAULT | Py_TPFLAGS_HAVE_GC,  /* tp_flags */
+    0,                                        /* tp_doc */
+    (traverseproc) bitarrayiter_traverse,     /* tp_traverse */
+    0,                                        /* tp_clear */
+    0,                                        /* tp_richcompare */
+    0,                                        /* tp_weaklistoffset */
+    PyObject_SelfIter,                        /* tp_iter */
+    (iternextfunc) bitarrayiter_next,         /* tp_iternext */
+    0,                                        /* tp_methods */
+};
+
+/********************* Bitarray Buffer Interface ************************/
+#ifdef WITH_BUFFER
+
+#if PY_MAJOR_VERSION == 2
+static Py_ssize_t
+bitarray_buffer_getreadbuf(bitarrayobject *self,
+                           Py_ssize_t index, const void **ptr)
+{
+    if (index != 0) {
+        PyErr_SetString(PyExc_SystemError, "accessing non-existent segment");
+        return -1;
+    }
+    *ptr = (void *) self->ob_item;
+    return Py_SIZE(self);
+}
+
+static Py_ssize_t
+bitarray_buffer_getwritebuf(bitarrayobject *self,
+                            Py_ssize_t index, const void **ptr)
+{
+    if (index != 0) {
+        PyErr_SetString(PyExc_SystemError, "accessing non-existent segment");
+        return -1;
+    }
+    *ptr = (void *) self->ob_item;
+    return Py_SIZE(self);
+}
+
+static Py_ssize_t
+bitarray_buffer_getsegcount(bitarrayobject *self, Py_ssize_t *lenp)
+{
+    if (lenp)
+        *lenp = Py_SIZE(self);
+    return 1;
+}
+
+static Py_ssize_t
+bitarray_buffer_getcharbuf(bitarrayobject *self,
+                           Py_ssize_t index, const char **ptr)
+{
+    if (index != 0) {
+        PyErr_SetString(PyExc_SystemError, "accessing non-existent segment");
+        return -1;
+    }
+    *ptr = self->ob_item;
+    return Py_SIZE(self);
+}
+
+#endif
+
+static int
+bitarray_getbuffer(bitarrayobject *self, Py_buffer *view, int flags)
+{
+    int ret;
+    void *ptr;
+
+    if (view == NULL) {
+        self->ob_exports++;
+        return 0;
+    }
+    ptr = (void *) self->ob_item;
+    ret = PyBuffer_FillInfo(view, (PyObject *) self, ptr,
+                            Py_SIZE(self), 0, flags);
+    if (ret >= 0) {
+        self->ob_exports++;
+    }
+    return ret;
+}
+
+static void
+bitarray_releasebuffer(bitarrayobject *self, Py_buffer *view)
+{
+    self->ob_exports--;
+}
+
+static PyBufferProcs bitarray_as_buffer = {
+#if PY_MAJOR_VERSION == 2   /* old buffer protocol */
+    (readbufferproc) bitarray_buffer_getreadbuf,
+    (writebufferproc) bitarray_buffer_getwritebuf,
+    (segcountproc) bitarray_buffer_getsegcount,
+    (charbufferproc) bitarray_buffer_getcharbuf,
+#endif
+    (getbufferproc) bitarray_getbuffer,
+    (releasebufferproc) bitarray_releasebuffer,
+};
+
+#endif  /* WITH_BUFFER */
+
+/************************** Bitarray Type *******************************/
+
+static PyTypeObject Bitarraytype = {
+#ifdef IS_PY3K
+    PyVarObject_HEAD_INIT(&Bitarraytype, 0)
+#else
+    PyObject_HEAD_INIT(NULL)
+    0,                                        /* ob_size */
+#endif
+    "bitarray._bitarray",                     /* tp_name */
+    sizeof(bitarrayobject),                   /* tp_basicsize */
+    0,                                        /* tp_itemsize */
+    /* methods */
+    (destructor) bitarray_dealloc,            /* tp_dealloc */
+    0,                                        /* tp_print */
+    0,                                        /* tp_getattr */
+    0,                                        /* tp_setattr */
+    0,                                        /* tp_compare */
+    (reprfunc) bitarray_repr,                 /* tp_repr */
+    0,                                        /* tp_as_number*/
+    0,                                        /* tp_as_sequence */
+    0,                                        /* tp_as_mapping */
+    0,                                        /* tp_hash */
+    0,                                        /* tp_call */
+    0,                                        /* tp_str */
+    PyObject_GenericGetAttr,                  /* tp_getattro */
+    0,                                        /* tp_setattro */
+#ifdef WITH_BUFFER
+    &bitarray_as_buffer,                      /* tp_as_buffer */
+#else
+    0,                                        /* tp_as_buffer */
+#endif
+    Py_TPFLAGS_DEFAULT | Py_TPFLAGS_BASETYPE | Py_TPFLAGS_HAVE_WEAKREFS
+#if defined(WITH_BUFFER) && PY_MAJOR_VERSION == 2
+    | Py_TPFLAGS_HAVE_NEWBUFFER
+#endif
+    ,                                         /* tp_flags */
+    0,                                        /* tp_doc */
+    0,                                        /* tp_traverse */
+    0,                                        /* tp_clear */
+    richcompare,                              /* tp_richcompare */
+    offsetof(bitarrayobject, weakreflist),    /* tp_weaklistoffset */
+    (getiterfunc) bitarray_iter,              /* tp_iter */
+    0,                                        /* tp_iternext */
+    bitarray_methods,                         /* tp_methods */
+    0,                                        /* tp_members */
+    0,                                        /* tp_getset */
+    0,                                        /* tp_base */
+    0,                                        /* tp_dict */
+    0,                                        /* tp_descr_get */
+    0,                                        /* tp_descr_set */
+    0,                                        /* tp_dictoffset */
+    0,                                        /* tp_init */
+    PyType_GenericAlloc,                      /* tp_alloc */
+    bitarray_new,                             /* tp_new */
+    PyObject_Del,                             /* tp_free */
+};
+
+/*************************** Module functions **********************/
+
+static PyObject *
+bitdiff(PyObject *self, PyObject *args)
+{
+    PyObject *a, *b;
+    Py_ssize_t i;
+    idx_t res = 0;
+    unsigned char c;
+
+    if (!PyArg_ParseTuple(args, "OO:bitdiff", &a, &b))
+        return NULL;
+    if (!(bitarray_Check(a) && bitarray_Check(b))) {
+        PyErr_SetString(PyExc_TypeError, "bitarray object expected");
+        return NULL;
+    }
+
+#define aa  ((bitarrayobject *) a)
+#define bb  ((bitarrayobject *) b)
+    if (aa->nbits != bb->nbits) {
+        PyErr_SetString(PyExc_ValueError,
+                        "bitarrays of equal length expected");
+        return NULL;
+    }
+    setunused(aa);
+    setunused(bb);
+    for (i = 0; i < Py_SIZE(aa); i++) {
+        c = aa->ob_item[i] ^ bb->ob_item[i];
+        res += bitcount_lookup[c];
+    }
+#undef aa
+#undef bb
+    return PyLong_FromLongLong(res);
+}
+
+PyDoc_STRVAR(bitdiff_doc,
+"bitdiff(a, b) -> int\n\
+\n\
+Return the difference between two bitarrays a and b.\n\
+This is function does the same as (a ^ b).count(), but is more memory\n\
+efficient, as no intermediate bitarray object gets created");
+
+
+static PyObject *
+bits2bytes(PyObject *self, PyObject *v)
+{
+    idx_t n = 0;
+
+    if (!IS_INDEX(v)) {
+        PyErr_SetString(PyExc_TypeError, "integer expected");
+        return NULL;
+    }
+    if (getIndex(v, &n) < 0)
+        return NULL;
+    if (n < 0) {
+        PyErr_SetString(PyExc_ValueError, "positive value expected");
+        return NULL;
+    }
+    return PyLong_FromLongLong(BYTES(n));
+}
+
+PyDoc_STRVAR(bits2bytes_doc,
+"bits2bytes(n) -> int\n\
+\n\
+Return the number of bytes necessary to store n bits.");
+
+
+static PyObject *
+sysinfo(void)
+{
+    return Py_BuildValue("iiiiL",
+                         (int) sizeof(void *),
+                         (int) sizeof(size_t),
+                         (int) sizeof(Py_ssize_t),
+                         (int) sizeof(idx_t),
+                         (idx_t) PY_SSIZE_T_MAX);
+}
+
+PyDoc_STRVAR(sysinfo_doc,
+"_sysinfo() -> tuple\n\
+\n\
+tuple(sizeof(void *),\n\
+      sizeof(size_t),\n\
+      sizeof(Py_ssize_t),\n\
+      sizeof(idx_t),\n\
+      PY_SSIZE_T_MAX)");
+
+
+static PyMethodDef module_functions[] = {
+    {"bitdiff",    (PyCFunction) bitdiff,    METH_VARARGS, bitdiff_doc   },
+    {"bits2bytes", (PyCFunction) bits2bytes, METH_O,       bits2bytes_doc},
+    {"_sysinfo",   (PyCFunction) sysinfo,    METH_NOARGS,  sysinfo_doc   },
+    {NULL,         NULL}  /* sentinel */
+};
+
+/*********************** Install Module **************************/
+
+#ifdef IS_PY3K
+static PyModuleDef moduledef = {
+    PyModuleDef_HEAD_INIT, "_bitarray", 0, -1, module_functions,
+};
+PyMODINIT_FUNC
+PyInit__bitarray(void)
+#else
+PyMODINIT_FUNC
+init_bitarray(void)
+#endif
+{
+    PyObject *m;
+
+    Py_TYPE(&Bitarraytype) = &PyType_Type;
+    Py_TYPE(&SearchIter_Type) = &PyType_Type;
+    Py_TYPE(&DecodeIter_Type) = &PyType_Type;
+    Py_TYPE(&BitarrayIter_Type) = &PyType_Type;
+#ifdef IS_PY3K
+    m = PyModule_Create(&moduledef);
+    if (m == NULL)
+        return NULL;
+#else
+    m = Py_InitModule3("_bitarray", module_functions, 0);
+    if (m == NULL)
+        return;
+#endif
+
+    Py_INCREF((PyObject *) &Bitarraytype);
+    PyModule_AddObject(m, "_bitarray", (PyObject *) &Bitarraytype);
+#ifdef IS_PY3K
+    return m;
+#endif
+}
diff --git a/shell/ext-py/bitarray-0.9.0/bitarray/test_bitarray.py b/shell/ext-py/bitarray-0.9.0/bitarray/test_bitarray.py
new file mode 100644
index 0000000..3e66bff
--- /dev/null
+++ b/shell/ext-py/bitarray-0.9.0/bitarray/test_bitarray.py
@@ -0,0 +1,2196 @@
+"""
+Tests for bitarray
+
+Author: Ilan Schnell
+"""
+import os
+import sys
+import unittest
+import tempfile
+import shutil
+from random import randint
+
+is_py3k = bool(sys.version_info[0] == 3)
+
+if is_py3k:
+    from io import StringIO
+else:
+    from cStringIO import StringIO
+
+
+from bitarray import bitarray, bitdiff, bits2bytes, __version__
+
+
+tests = []
+
+if sys.version_info[:2] < (2, 6):
+    def next(x):
+        return x.next()
+
+
+def to_bytes(s):
+    if is_py3k:
+        return bytes(s.encode('latin1'))
+    elif sys.version_info[:2] >= (2, 6):
+        return bytes(s)
+    else:
+        return s
+
+
+class Util(object):
+
+    def randombitarrays(self):
+        for n in list(range(25)) + [randint(1000, 2000)]:
+            a = bitarray(endian=['little', 'big'][randint(0, 1)])
+            a.frombytes(os.urandom(bits2bytes(n)))
+            del a[n:]
+            yield a
+
+    def randomlists(self):
+        for n in list(range(25)) + [randint(1000, 2000)]:
+            yield [bool(randint(0, 1)) for d in range(n)]
+
+    def rndsliceidx(self, length):
+        if randint(0, 1):
+            return None
+        else:
+            return randint(-2 * length, 2 * length - 1)
+
+    def slicelen(self, r, length):
+        return getIndicesEx(r, length)[-1]
+
+    def check_obj(self, a):
+        self.assertEqual(repr(type(a)), "<class 'bitarray.bitarray'>")
+        unused = 8 * a.buffer_info()[1] - len(a)
+        self.assert_(0 <= unused < 8)
+        self.assertEqual(unused, a.buffer_info()[3])
+
+    def assertEQUAL(self, a, b):
+        self.assertEqual(a, b)
+        self.assertEqual(a.endian(), b.endian())
+        self.check_obj(a)
+        self.check_obj(b)
+
+    def assertStopIteration(self, it):
+        if is_py3k:
+            return
+        self.assertRaises(StopIteration, it.next)
+
+
+def getIndicesEx(r, length):
+    if not isinstance(r, slice):
+        raise TypeError("slice object expected")
+    start = r.start
+    stop  = r.stop
+    step  = r.step
+    if r.step is None:
+        step = 1
+    else:
+        if step == 0:
+            raise ValueError("slice step cannot be zero")
+
+    if step < 0:
+        defstart = length - 1
+        defstop = -1
+    else:
+        defstart = 0
+        defstop = length
+
+    if r.start is None:
+        start = defstart
+    else:
+        if start < 0: start += length
+        if start < 0: start = [0, -1][step < 0]
+        if start >= length: start = [length, length - 1][step < 0]
+
+    if r.stop is None:
+        stop = defstop
+    else:
+        if stop < 0: stop += length
+        if stop < 0: stop = -1
+        if stop > length: stop = length
+
+    if (step < 0 and stop >= length) or (step > 0 and start >= stop):
+        slicelength = 0
+    elif step < 0:
+        slicelength = (stop - start + 1) / step + 1
+    else:
+        slicelength = (stop - start - 1) / step + 1
+
+    if slicelength < 0:
+        slicelength = 0
+
+    return start, stop, step, slicelength
+
+# ---------------------------------------------------------------------------
+
+class TestsModuleFunctions(unittest.TestCase, Util):
+
+    def test_bitdiff(self):
+        a = bitarray('0011')
+        b = bitarray('0101')
+        self.assertEqual(bitdiff(a, b), 2)
+        self.assertRaises(TypeError, bitdiff, a, '')
+        self.assertRaises(TypeError, bitdiff, '1', b)
+        self.assertRaises(TypeError, bitdiff, a, 4)
+        b.append(1)
+        self.assertRaises(ValueError, bitdiff, a, b)
+
+        for n in list(range(50)) + [randint(1000, 2000)]:
+            a = bitarray()
+            a.frombytes(os.urandom(bits2bytes(n)))
+            del a[n:]
+            b = bitarray()
+            b.frombytes(os.urandom(bits2bytes(n)))
+            del b[n:]
+            diff = sum(a[i] ^ b[i] for i in range(n))
+            self.assertEqual(bitdiff(a, b), diff)
+
+    def test_bits2bytes(self):
+        for arg in ['foo', [], None, {}]:
+            self.assertRaises(TypeError, bits2bytes, arg)
+
+        self.assertRaises(TypeError, bits2bytes)
+        self.assertRaises(TypeError, bits2bytes, 1, 2)
+
+        self.assertRaises(ValueError, bits2bytes, -1)
+        self.assertRaises(ValueError, bits2bytes, -924)
+
+        self.assertEqual(bits2bytes(0), 0)
+        for n in range(1, 1000):
+            self.assertEqual(bits2bytes(n), (n - 1) // 8 + 1)
+
+        for n, m in [(0, 0), (1, 1), (2, 1), (7, 1), (8, 1), (9, 2),
+                     (10, 2), (15, 2), (16, 2), (64, 8), (65, 9),
+                     (0, 0), (1, 1), (65, 9), (2**29, 2**26),
+                     (2**31, 2**28), (2**32, 2**29), (2**34, 2**31),
+                     (2**34+793, 2**31+100), (2**35-8, 2**32-1),
+                     (2**62, 2**59), (2**63-8, 2**60-1)]:
+            self.assertEqual(bits2bytes(n), m)
+
+
+tests.append(TestsModuleFunctions)
+
+# ---------------------------------------------------------------------------
+
+class CreateObjectTests(unittest.TestCase, Util):
+
+    def test_noInitializer(self):
+        a = bitarray()
+        self.assertEqual(len(a), 0)
+        self.assertEqual(a.tolist(), [])
+        self.check_obj(a)
+
+    def test_endian1(self):
+        a = bitarray(endian='little')
+        a.fromstring('A')
+        self.assertEqual(a.endian(), 'little')
+        self.check_obj(a)
+
+        b = bitarray(endian='big')
+        b.fromstring('A')
+        self.assertEqual(b.endian(), 'big')
+        self.check_obj(b)
+
+        self.assertEqual(a.tostring(), b.tostring())
+
+    def test_endian2(self):
+        a = bitarray(endian='little')
+        a.fromstring(' ')
+        self.assertEqual(a.endian(), 'little')
+        self.check_obj(a)
+
+        b = bitarray(endian='big')
+        b.fromstring(' ')
+        self.assertEqual(b.endian(), 'big')
+        self.check_obj(b)
+
+        self.assertEqual(a.tostring(), b.tostring())
+
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, endian=0)
+        self.assertRaises(ValueError, bitarray.__new__, bitarray, endian='')
+        self.assertRaises(ValueError, bitarray.__new__,
+                          bitarray, endian='foo')
+
+    def test_integers(self):
+        for n in range(50):
+            a = bitarray(n)
+            self.assertEqual(len(a), n)
+            self.check_obj(a)
+
+            a = bitarray(int(n))
+            self.assertEqual(len(a), n)
+            self.check_obj(a)
+
+        self.assertRaises(ValueError, bitarray.__new__, bitarray, -1)
+        self.assertRaises(ValueError, bitarray.__new__, bitarray, -924)
+
+    def test_list(self):
+        lst = ['foo', None, [1], {}]
+        a = bitarray(lst)
+        self.assertEqual(a.tolist(), [True, False, True, False])
+        self.check_obj(a)
+
+        for n in range(50):
+            lst = [bool(randint(0, 1)) for d in range(n)]
+            a = bitarray(lst)
+            self.assertEqual(a.tolist(), lst)
+            self.check_obj(a)
+
+    def test_tuple(self):
+        tup = ('', True, [], {1:2})
+        a = bitarray(tup)
+        self.assertEqual(a.tolist(), [False, True, False, True])
+        self.check_obj(a)
+
+        for n in range(50):
+            lst = [bool(randint(0, 1)) for d in range(n)]
+            a = bitarray(tuple(lst))
+            self.assertEqual(a.tolist(), lst)
+            self.check_obj(a)
+
+    def test_iter1(self):
+        for n in range(50):
+            lst = [bool(randint(0, 1)) for d in range(n)]
+            a = bitarray(iter(lst))
+            self.assertEqual(a.tolist(), lst)
+            self.check_obj(a)
+
+    def test_iter2(self):
+        for lst in self.randomlists():
+            def foo():
+                for x in lst:
+                    yield x
+            a = bitarray(foo())
+            self.assertEqual(a, bitarray(lst))
+            self.check_obj(a)
+
+    def test_01(self):
+        a = bitarray('0010111')
+        self.assertEqual(a.tolist(), [0, 0, 1, 0, 1, 1, 1])
+        self.check_obj(a)
+
+        for n in range(50):
+            lst = [bool(randint(0, 1)) for d in range(n)]
+            s = ''.join([['0', '1'][x] for x in lst])
+            a = bitarray(s)
+            self.assertEqual(a.tolist(), lst)
+            self.check_obj(a)
+
+        self.assertRaises(ValueError, bitarray.__new__, bitarray, '01012100')
+
+    def test_rawbytes(self): # this representation is used for pickling
+        for s, r in [('\x00', ''), ('\x07\xff', '1'), ('\x03\xff', '11111'),
+                     ('\x01\x87\xda', '10000111' '1101101')]:
+            self.assertEqual(bitarray(to_bytes(s), endian='big'),
+                             bitarray(r))
+
+        for i in range(1, 8):
+            self.assertRaises(ValueError, bitarray.__new__,
+                              bitarray, to_bytes(chr(i)))
+
+    def test_bitarray(self):
+        for n in range(50):
+            a = bitarray(n)
+            b = bitarray(a)
+            self.assert_(a is not b)
+            self.assertEQUAL(a, b)
+
+        for end in ('little', 'big'):
+            a = bitarray(endian=end)
+            c = bitarray(a)
+            self.assertEqual(c.endian(), end)
+            c = bitarray(a, endian='little')
+            self.assertEqual(c.endian(), 'little')
+            c = bitarray(a, endian='big')
+            self.assertEqual(c.endian(), 'big')
+
+
+    def test_None(self):
+        self.assertEQUAL(bitarray(), bitarray(0))
+        self.assertEQUAL(bitarray(), bitarray(None))
+
+
+    def test_WrongArgs(self):
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, 'A', 42, 69)
+
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, Ellipsis)
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, slice(0))
+
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, 2.345)
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, 4+3j)
+
+        self.assertRaises(TypeError, bitarray.__new__, bitarray, '', 0, 42)
+        self.assertRaises(ValueError, bitarray.__new__, bitarray, 0, 'foo')
+
+
+tests.append(CreateObjectTests)
+
+# ---------------------------------------------------------------------------
+
+class ToObjectsTests(unittest.TestCase, Util):
+
+    def test_int(self):
+        a = bitarray()
+        self.assertRaises(TypeError, int, a)
+        if not is_py3k:
+            self.assertRaises(TypeError, long, a)
+
+    def test_float(self):
+        a = bitarray()
+        self.assertRaises(TypeError, float, a)
+
+    def test_complext(self):
+        a = bitarray()
+        self.assertRaises(TypeError, complex, a)
+
+    def test_list(self):
+        for a in self.randombitarrays():
+            self.assertEqual(list(a), a.tolist())
+
+    def test_tuple(self):
+        for a in self.randombitarrays():
+            self.assertEqual(tuple(a), tuple(a.tolist()))
+
+
+tests.append(ToObjectsTests)
+
+# ---------------------------------------------------------------------------
+
+class MetaDataTests(unittest.TestCase):
+
+    def test_buffer_info1(self):
+        a = bitarray('0000111100001', endian='little')
+        self.assertEqual(a.buffer_info()[1:4], (2, 'little', 3))
+
+        a = bitarray()
+        self.assertRaises(TypeError, a.buffer_info, 42)
+
+        bi = a.buffer_info()
+        self.assert_(isinstance(bi, tuple))
+        self.assertEqual(len(bi), 5)
+
+        self.assert_(isinstance(bi[0], int))
+        if is_py3k:
+            self.assert_(isinstance(bi[1], int))
+        self.assert_(isinstance(bi[2], str))
+        self.assert_(isinstance(bi[3], int))
+        if is_py3k:
+            self.assert_(isinstance(bi[4], int))
+
+    def test_buffer_info2(self):
+        for n in range(50):
+            bi = bitarray(n).buffer_info()
+            self.assertEqual(bi[1], bits2bytes(n))
+            self.assertEqual(bi[3] + n, 8 * bi[1])
+            self.assert_(bi[4] >= bi[1])
+
+    def test_buffer_info3(self):
+        a = bitarray(endian='little')
+        self.assertEqual(a.buffer_info()[2], 'little')
+
+        a = bitarray(endian='big')
+        self.assertEqual(a.buffer_info()[2], 'big')
+
+
+    def test_endian(self):
+        a = bitarray(endian='little')
+        self.assertEqual(a.endian(), 'little')
+
+        a = bitarray(endian='big')
+        self.assertEqual(a.endian(), 'big')
+
+
+    def test_length(self):
+        for n in range(1000):
+            a = bitarray(n)
+            self.assertEqual(len(a), n)
+            self.assertEqual(a.length(), n)
+
+
+tests.append(MetaDataTests)
+
+# ---------------------------------------------------------------------------
+
+class SliceTests(unittest.TestCase, Util):
+
+    def test_getitem1(self):
+        a = bitarray()
+        self.assertRaises(IndexError, a.__getitem__,  0)
+        a.append(True)
+        self.assertEqual(a[0], True)
+        self.assertRaises(IndexError, a.__getitem__,  1)
+        self.assertRaises(IndexError, a.__getitem__, -2)
+
+        a.append(False)
+        self.assertEqual(a[1], False)
+        self.assertRaises(IndexError, a.__getitem__,  2)
+        self.assertRaises(IndexError, a.__getitem__, -3)
+
+    def test_getitem2(self):
+        a = bitarray('1100010')
+        for i, b in enumerate([True, True, False, False, False, True, False]):
+            self.assertEqual(a[i], b)
+            self.assertEqual(a[i-7], b)
+        self.assertRaises(IndexError, a.__getitem__,  7)
+        self.assertRaises(IndexError, a.__getitem__, -8)
+
+    def test_getitem3(self):
+        a = bitarray('0100000100001')
+        self.assertEQUAL(a[:], a)
+        self.assert_(a[:] is not a)
+        aa = a.tolist()
+        self.assertEQUAL(a[11:2:-3], bitarray(aa[11:2:-3]))
+        self.check_obj(a[:])
+
+        self.assertRaises(ValueError, a.__getitem__, slice(None, None, 0))
+        self.assertRaises(TypeError, a.__getitem__, (1, 2))
+
+    def test_getitem4(self):
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            la = len(a)
+            if la == 0: continue
+            for dum in range(10):
+                step = self.rndsliceidx(la)
+                if step == 0: step = None
+                s = slice(self.rndsliceidx(la),
+                          self.rndsliceidx(la), step)
+                self.assertEQUAL(a[s], bitarray(aa[s], endian=a.endian()))
+
+    def test_setitem1(self):
+        a = bitarray([False])
+        a[0] = 1
+        self.assertEqual(a.tolist(), [True])
+
+        a = bitarray(2)
+        a[0] = 0
+        a[1] = 1
+        self.assertEqual(a.tolist(), [False, True])
+        a[-1] = 0
+        a[-2] = 1
+        self.assertEqual(a.tolist(), [True, False])
+
+        self.assertRaises(IndexError, a.__setitem__,  2, True)
+        self.assertRaises(IndexError, a.__setitem__, -3, False)
+
+    def test_setitem2(self):
+        for a in self.randombitarrays():
+            la = len(a)
+            if la == 0:
+                continue
+            i = randint(0, la - 1)
+            aa = a.tolist()
+            ida = id(a)
+            val = bool(randint(0, 1))
+            a[i] = val
+            aa[i] = val
+            self.assertEqual(a.tolist(), aa)
+            self.assertEqual(id(a), ida)
+            self.check_obj(a)
+
+            b = bitarray(la)
+            b[0:la] = bitarray(a)
+            self.assertEqual(a, b)
+            self.assertNotEqual(id(a), id(b))
+
+            b = bitarray(la)
+            b[:] = bitarray(a)
+            self.assertEqual(a, b)
+            self.assertNotEqual(id(a), id(b))
+
+            b = bitarray(la)
+            b[::-1] = bitarray(a)
+            self.assertEqual(a.tolist()[::-1], b.tolist())
+
+    def test_setitem3(self):
+        a = bitarray(5 * [False])
+        a[0] = 1
+        a[-2] = 1
+        self.assertEqual(a, bitarray('10010'))
+        self.assertRaises(IndexError, a.__setitem__,  5, 'foo')
+        self.assertRaises(IndexError, a.__setitem__, -6, 'bar')
+
+    def test_setitem4(self):
+        for a in self.randombitarrays():
+            la = len(a)
+            if la == 0: continue
+            for dum in range(50):
+                step = self.rndsliceidx(la)
+                if step == 0: step = None
+                s = slice(self.rndsliceidx(la),
+                          self.rndsliceidx(la), step)
+                for b in self.randombitarrays():
+                    if len(b) == self.slicelen(s, len(a)) or step is None:
+                        c = bitarray(a)
+                        d = c
+                        c[s] = b
+                        self.assert_(c is d)
+                        self.check_obj(c)
+                        cc = a.tolist()
+                        cc[s] = b.tolist()
+                        self.assertEqual(c, bitarray(cc))
+
+
+    def test_setslice_to_bool(self):
+        a = bitarray('11111111')
+        a[::2] = False
+        self.assertEqual(a, bitarray('01010101'))
+        a[4::] = True
+        self.assertEqual(a, bitarray('01011111'))
+        a[-2:] = False
+        self.assertEqual(a, bitarray('01011100'))
+        a[:2:] = True
+        self.assertEqual(a, bitarray('11011100'))
+        a[:] = True
+        self.assertEqual(a, bitarray('11111111'))
+
+    def test_setslice_to_int(self):
+        a = bitarray('11111111')
+        a[::2] = 0
+        self.assertEqual(a, bitarray('01010101'))
+        a[4::] = 1
+        self.assertEqual(a, bitarray('01011111'))
+        a.__setitem__(slice(-2, None, None), 0)
+        self.assertEqual(a, bitarray('01011100'))
+
+        self.assertRaises(ValueError, a.__setitem__,
+                          slice(None, None, 2), 3)
+        self.assertRaises(ValueError, a.__setitem__,
+                          slice(None, 2, None), -1)
+
+
+    def test_delitem1(self):
+        a = bitarray('100110')
+        del a[1]
+        self.assertEqual(len(a), 5)
+        del a[3]
+        del a[-2]
+        self.assertEqual(a, bitarray('100'))
+        self.assertRaises(IndexError, a.__delitem__,  3)
+        self.assertRaises(IndexError, a.__delitem__, -4)
+
+    def test_delitem2(self):
+        for a in self.randombitarrays():
+            la = len(a)
+            if la == 0: continue
+            for dum in range(50):
+                step = self.rndsliceidx(la)
+                if step == 0: step = None
+                s = slice(self.rndsliceidx(la),
+                          self.rndsliceidx(la), step)
+                c = bitarray(a)
+                d = c
+                del c[s]
+                self.assert_(c is d)
+                self.check_obj(c)
+                cc = a.tolist()
+                del cc[s]
+                self.assertEQUAL(c, bitarray(cc, endian=c.endian()))
+
+
+tests.append(SliceTests)
+
+# ---------------------------------------------------------------------------
+
+class MiscTests(unittest.TestCase, Util):
+
+    def test_instancecheck(self):
+        a = bitarray('011')
+        self.assertTrue(isinstance(a, bitarray))
+        self.assertFalse(isinstance(a, str))
+
+    def test_booleanness(self):
+        self.assertEqual(bool(bitarray('')), False)
+        self.assertEqual(bool(bitarray('0')), True)
+        self.assertEqual(bool(bitarray('1')), True)
+
+    def test_iterate(self):
+        for lst in self.randomlists():
+            acc = []
+            for b in bitarray(lst):
+                acc.append(b)
+            self.assertEqual(acc, lst)
+
+    def test_iter1(self):
+        it = iter(bitarray('011'))
+        self.assertEqual(next(it), False)
+        self.assertEqual(next(it), True)
+        self.assertEqual(next(it), True)
+        self.assertStopIteration(it)
+
+    def test_iter2(self):
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            self.assertEqual(list(a), aa)
+            self.assertEqual(list(iter(a)), aa)
+
+    def test_assignment(self):
+        a = bitarray('00110111001')
+        a[1:3] = a[7:9]
+        a[-1:] = a[:1]
+        b = bitarray('01010111000')
+        self.assertEqual(a, b)
+
+    def test_compare(self):
+        for a in self.randombitarrays():
+            aa = a.tolist()
+
+            for b in self.randombitarrays():
+                bb = b.tolist()
+                self.assertEqual(a == b, aa == bb)
+                self.assertEqual(a != b, aa != bb)
+                self.assertEqual(a <= b, aa <= bb)
+                self.assertEqual(a <  b, aa <  bb)
+                self.assertEqual(a >= b, aa >= bb)
+                self.assertEqual(a >  b, aa >  bb)
+
+    def test_subclassing(self):
+        class ExaggeratingBitarray(bitarray):
+
+            def __new__(cls, data, offset):
+                return bitarray.__new__(cls, data)
+
+            def __init__(self, data, offset):
+                self.offset = offset
+
+            def __getitem__(self, i):
+                return bitarray.__getitem__(self, i - self.offset)
+
+        for a in self.randombitarrays():
+            if len(a) == 0:
+                continue
+            b = ExaggeratingBitarray(a, 1234)
+            for i in range(len(a)):
+                self.assertEqual(a[i], b[i+1234])
+
+    def test_endianness1(self):
+        a = bitarray(endian='little')
+        a.frombytes(to_bytes('\x01'))
+        self.assertEqual(a.to01(), '10000000')
+
+        b = bitarray(endian='little')
+        b.frombytes(to_bytes('\x80'))
+        self.assertEqual(b.to01(), '00000001')
+
+        c = bitarray(endian='big')
+        c.frombytes(to_bytes('\x80'))
+        self.assertEqual(c.to01(), '10000000')
+
+        d = bitarray(endian='big')
+        d.frombytes(to_bytes('\x01'))
+        self.assertEqual(d.to01(), '00000001')
+
+        self.assertEqual(a, c)
+        self.assertEqual(b, d)
+
+    def test_endianness2(self):
+        a = bitarray(8, endian='little')
+        a.setall(False)
+        a[0] = True
+        self.assertEqual(a.tobytes(), to_bytes('\x01'))
+        a[1] = True
+        self.assertEqual(a.tobytes(), to_bytes('\x03'))
+        a.frombytes(to_bytes(' '))
+        self.assertEqual(a.tobytes(), to_bytes('\x03 '))
+        self.assertEqual(a.to01(), '1100000000000100')
+
+    def test_endianness3(self):
+        a = bitarray(8, endian='big')
+        a.setall(False)
+        a[7] = True
+        self.assertEqual(a.tobytes(), to_bytes('\x01'))
+        a[6] = True
+        self.assertEqual(a.tobytes(), to_bytes('\x03'))
+        a.frombytes(to_bytes(' '))
+        self.assertEqual(a.tobytes(), to_bytes('\x03 '))
+        self.assertEqual(a.to01(), '0000001100100000')
+
+    def test_endianness4(self):
+        a = bitarray('00100000', endian='big')
+        self.assertEqual(a.tobytes(), to_bytes(' '))
+        b = bitarray('00000100', endian='little')
+        self.assertEqual(b.tobytes(), to_bytes(' '))
+        self.assertNotEqual(a, b)
+
+    def test_endianness5(self):
+        a = bitarray('11100000', endian='little')
+        b = bitarray(a, endian='big')
+        self.assertNotEqual(a, b)
+        self.assertEqual(a.tobytes(), b.tobytes())
+
+    def test_pickle(self):
+        from pickle import loads, dumps
+        for v in range(3):
+            for a in self.randombitarrays():
+                b = loads(dumps(a, v))
+                self.assert_(b is not a)
+                self.assertEQUAL(a, b)
+
+    def test_cPickle(self):
+        if is_py3k:
+            return
+        for v in range(3):
+            from cPickle import loads, dumps
+            for a in self.randombitarrays():
+                b = loads(dumps(a, v))
+                self.assert_(b is not a)
+                self.assertEQUAL(a, b)
+
+    def test_overflow(self):
+        from platform import architecture
+
+        if architecture()[0] == '64bit':
+            return
+
+        self.assertRaises(OverflowError, bitarray.__new__,
+                          bitarray, 2**34 + 1)
+
+        a = bitarray(10 ** 6)
+        self.assertRaises(OverflowError, a.__imul__, 17180)
+
+
+tests.append(MiscTests)
+
+# ---------------------------------------------------------------------------
+
+class SpecialMethodTests(unittest.TestCase, Util):
+
+    def test_all(self):
+        a = bitarray()
+        self.assertTrue(a.all())
+
+        if sys.version_info[:2] < (2, 5):
+            return
+
+        for a in self.randombitarrays():
+            self.assertEqual(all(a),          a.all())
+            self.assertEqual(all(a.tolist()), a.all())
+
+
+    def test_any(self):
+        a = bitarray()
+        self.assertFalse(a.any())
+
+        if sys.version_info[:2] < (2, 5):
+            return
+
+        for a in self.randombitarrays():
+            self.assertEqual(any(a),          a.any())
+            self.assertEqual(any(a.tolist()), a.any())
+
+
+    def test_repr(self):
+        a = bitarray()
+        self.assertEqual(repr(a), "bitarray()")
+
+        a = bitarray('10111')
+        self.assertEqual(repr(a), "bitarray('10111')")
+
+        for a in self.randombitarrays():
+            b = eval(repr(a))
+            self.assert_(b is not a)
+            self.assertEqual(a, b)
+            self.check_obj(b)
+
+
+    def test_copy(self):
+        import copy
+        for a in self.randombitarrays():
+            b = a.copy()
+            self.assert_(b is not a)
+            self.assertEQUAL(a, b)
+
+            b = copy.copy(a)
+            self.assert_(b is not a)
+            self.assertEQUAL(a, b)
+
+            b = copy.deepcopy(a)
+            self.assert_(b is not a)
+            self.assertEQUAL(a, b)
+
+
+    def assertReallyEqual(self, a, b):
+        # assertEqual first, because it will have a good message if the
+        # assertion fails.
+        self.assertEqual(a, b)
+        self.assertEqual(b, a)
+        self.assertTrue(a == b)
+        self.assertTrue(b == a)
+        self.assertFalse(a != b)
+        self.assertFalse(b != a)
+        if not is_py3k:
+            self.assertEqual(0, cmp(a, b))
+            self.assertEqual(0, cmp(b, a))
+
+    def assertReallyNotEqual(self, a, b):
+        # assertNotEqual first, because it will have a good message if the
+        # assertion fails.
+        self.assertNotEqual(a, b)
+        self.assertNotEqual(b, a)
+        self.assertFalse(a == b)
+        self.assertFalse(b == a)
+        self.assertTrue(a != b)
+        self.assertTrue(b != a)
+        if not is_py3k:
+            self.assertNotEqual(0, cmp(a, b))
+            self.assertNotEqual(0, cmp(b, a))
+
+    def test_equality(self):
+        self.assertReallyEqual(bitarray(''), bitarray(''))
+        self.assertReallyEqual(bitarray('0'), bitarray('0'))
+        self.assertReallyEqual(bitarray('1'), bitarray('1'))
+
+    def test_not_equality(self):
+        self.assertReallyNotEqual(bitarray(''), bitarray('1'))
+        self.assertReallyNotEqual(bitarray(''), bitarray('0'))
+        self.assertReallyNotEqual(bitarray('0'), bitarray('1'))
+
+tests.append(SpecialMethodTests)
+
+# ---------------------------------------------------------------------------
+
+class NumberTests(unittest.TestCase, Util):
+
+    def test_add(self):
+        c = bitarray('001') + bitarray('110')
+        self.assertEQUAL(c, bitarray('001110'))
+
+        for a in self.randombitarrays():
+            aa = a.copy()
+            for b in self.randombitarrays():
+                bb = b.copy()
+                c = a + b
+                self.assertEqual(c, bitarray(a.tolist() + b.tolist()))
+                self.assertEqual(c.endian(), a.endian())
+                self.check_obj(c)
+
+                self.assertEQUAL(a, aa)
+                self.assertEQUAL(b, bb)
+
+        a = bitarray()
+        self.assertRaises(TypeError, a.__add__, 42)
+
+
+    def test_iadd(self):
+        c = bitarray('001')
+        c += bitarray('110')
+        self.assertEQUAL(c, bitarray('001110'))
+
+        for a in self.randombitarrays():
+            for b in self.randombitarrays():
+                c = bitarray(a)
+                d = c
+                d += b
+                self.assertEqual(d, a + b)
+                self.assert_(c is d)
+                self.assertEQUAL(c, d)
+                self.assertEqual(d.endian(), a.endian())
+                self.check_obj(d)
+
+        a = bitarray()
+        self.assertRaises(TypeError, a.__iadd__, 42)
+
+
+    def test_mul(self):
+        c = 0 * bitarray('1001111')
+        self.assertEQUAL(c, bitarray())
+
+        c = 3 * bitarray('001')
+        self.assertEQUAL(c, bitarray('001001001'))
+
+        c = bitarray('110') * 3
+        self.assertEQUAL(c, bitarray('110110110'))
+
+        for a in self.randombitarrays():
+            b = a.copy()
+            for n in range(-10, 20):
+                c = a * n
+                self.assertEQUAL(c, bitarray(n * a.tolist(),
+                                             endian=a.endian()))
+                c = n * a
+                self.assertEqual(c, bitarray(n * a.tolist(),
+                                             endian=a.endian()))
+                self.assertEQUAL(a, b)
+
+        a = bitarray()
+        self.assertRaises(TypeError, a.__mul__, None)
+
+
+    def test_imul(self):
+        c = bitarray('1101110011')
+        idc = id(c)
+        c *= 0
+        self.assertEQUAL(c, bitarray())
+        self.assertEqual(idc, id(c))
+
+        c = bitarray('110')
+        c *= 3
+        self.assertEQUAL(c, bitarray('110110110'))
+
+        for a in self.randombitarrays():
+            for n in range(-10, 10):
+                b = a.copy()
+                idb = id(b)
+                b *= n
+                self.assertEQUAL(b, bitarray(n * a.tolist(),
+                                             endian=a.endian()))
+                self.assertEqual(idb, id(b))
+
+        a = bitarray()
+        self.assertRaises(TypeError, a.__imul__, None)
+
+
+tests.append(NumberTests)
+
+# ---------------------------------------------------------------------------
+
+class BitwiseTests(unittest.TestCase, Util):
+
+    def test_misc(self):
+        for a in self.randombitarrays():
+            b = ~a
+            c = a & b
+            self.assertEqual(c.any(), False)
+            self.assertEqual(a, a ^ c)
+            d = a ^ b
+            self.assertEqual(d.all(), True)
+            b &= d
+            self.assertEqual(~b, a)
+
+    def test_and(self):
+        a = bitarray('11001')
+        b = bitarray('10011')
+        self.assertEQUAL(a & b, bitarray('10001'))
+
+        b = bitarray('1001')
+        self.assertRaises(ValueError, a.__and__, b) # not same length
+
+        self.assertRaises(TypeError, a.__and__, 42)
+
+    def test_iand(self):
+        a =  bitarray('110010110')
+        ida = id(a)
+        a &= bitarray('100110011')
+        self.assertEQUAL(a, bitarray('100010010'))
+        self.assertEqual(ida, id(a))
+
+    def test_or(self):
+        a = bitarray('11001')
+        b = bitarray('10011')
+        self.assertEQUAL(a | b, bitarray('11011'))
+
+    def test_ior(self):
+        a =  bitarray('110010110')
+        a |= bitarray('100110011')
+        self.assertEQUAL(a, bitarray('110110111'))
+
+    def test_xor(self):
+        a = bitarray('11001')
+        b = bitarray('10011')
+        self.assertEQUAL(a ^ b, bitarray('01010'))
+
+    def test_ixor(self):
+        a =  bitarray('110010110')
+        a ^= bitarray('100110011')
+        self.assertEQUAL(a, bitarray('010100101'))
+
+    def test_invert(self):
+        a = bitarray()
+        a.invert()
+        self.assertEQUAL(a, bitarray())
+
+        a = bitarray('11011')
+        a.invert()
+        self.assertEQUAL(a, bitarray('00100'))
+
+        a = bitarray('11011')
+        b = ~a
+        self.assertEQUAL(b, bitarray('00100'))
+        self.assertEQUAL(a, bitarray('11011'))
+        self.assert_(a is not b)
+
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            b = bitarray(a)
+            b.invert()
+            for i in range(len(a)):
+                self.assertEqual(b[i], not aa[i])
+            self.check_obj(b)
+
+            c = ~a
+            self.assert_(c is not a)
+            self.assertEQUAL(a, bitarray(aa, endian=a.endian()))
+
+            for i in range(len(a)):
+                self.assertEqual(c[i], not aa[i])
+
+            self.check_obj(b)
+
+
+tests.append(BitwiseTests)
+
+# ---------------------------------------------------------------------------
+
+class SequenceTests(unittest.TestCase, Util):
+
+    def test_contains1(self):
+        a = bitarray()
+        self.assert_(False not in a)
+        self.assert_(True not in a)
+        self.assert_(bitarray() in a)
+        a.append(True)
+        self.assert_(True in a)
+        self.assert_(False not in a)
+        a = bitarray([False])
+        self.assert_(False in a)
+        self.assert_(True not in a)
+        a.append(True)
+        self.assert_(0 in a)
+        self.assert_(1 in a)
+        if not is_py3k:
+            self.assert_(long(0) in a)
+            self.assert_(long(1) in a)
+
+    def test_contains2(self):
+        a = bitarray()
+        self.assertEqual(a.__contains__(1), False)
+        a.append(1)
+        self.assertEqual(a.__contains__(1), True)
+        a = bitarray('0011')
+        self.assertEqual(a.__contains__(bitarray('01')), True)
+        self.assertEqual(a.__contains__(bitarray('10')), False)
+        self.assertRaises(TypeError, a.__contains__, 'asdf')
+        self.assertRaises(ValueError, a.__contains__, 2)
+        self.assertRaises(ValueError, a.__contains__, -1)
+        if not is_py3k:
+            self.assertRaises(ValueError, a.__contains__, long(2))
+
+    def test_contains3(self):
+        for n in range(2, 100):
+            a = bitarray(n)
+            a.setall(0)
+            self.assert_(False in a)
+            self.assert_(True not in a)
+            a[randint(0, n - 1)] = 1
+            self.assert_(True in a)
+            self.assert_(False in a)
+            a.setall(1)
+            self.assert_(True in a)
+            self.assert_(False not in a)
+            a[randint(0, n - 1)] = 0
+            self.assert_(True in a)
+            self.assert_(False in a)
+
+    def test_contains4(self):
+        a = bitarray('011010000001')
+        for s, r in [('', True), ('1', True), ('11', True), ('111', False),
+                     ('011', True), ('0001', True), ('00011', False)]:
+            self.assertEqual(bitarray(s) in a, r)
+
+
+tests.append(SequenceTests)
+
+# ---------------------------------------------------------------------------
+
+class ExtendTests(unittest.TestCase, Util):
+
+    def test_wrongArgs(self):
+        a = bitarray()
+        self.assertRaises(TypeError, a.extend)
+        self.assertRaises(TypeError, a.extend, None)
+        self.assertRaises(TypeError, a.extend, True)
+        self.assertRaises(TypeError, a.extend, 24)
+        self.assertRaises(ValueError, a.extend, '0011201')
+
+    def test_bitarray(self):
+        a = bitarray()
+        a.extend(bitarray())
+        self.assertEqual(a, bitarray())
+        a.extend(bitarray('110'))
+        self.assertEqual(a, bitarray('110'))
+        a.extend(bitarray('1110'))
+        self.assertEqual(a, bitarray('1101110'))
+
+        a = bitarray('00001111', endian='little')
+        a.extend(bitarray('00111100', endian='big'))
+        self.assertEqual(a, bitarray('0000111100111100'))
+
+        for a in self.randombitarrays():
+            for b in self.randombitarrays():
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(b)
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c, a + b)
+
+    def test_list(self):
+        a = bitarray()
+        a.extend([0, 1, 3, None, {}])
+        self.assertEqual(a, bitarray('01100'))
+        a.extend([True, False])
+        self.assertEqual(a, bitarray('0110010'))
+
+        for a in self.randomlists():
+            for b in self.randomlists():
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(b)
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c.tolist(), a + b)
+                self.check_obj(c)
+
+    def test_tuple(self):
+        a = bitarray()
+        a.extend((0, 1, 2, 0, 3))
+        self.assertEqual(a, bitarray('01101'))
+
+        for a in self.randomlists():
+            for b in self.randomlists():
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(tuple(b))
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c.tolist(), a + b)
+                self.check_obj(c)
+
+    def test_generator(self):
+        def bar():
+            for x in ('', '1', None, True, []):
+                yield x
+        a = bitarray()
+        a.extend(bar())
+        self.assertEqual(a, bitarray('01010'))
+
+        for a in self.randomlists():
+            for b in self.randomlists():
+                def foo():
+                    for e in b:
+                        yield e
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(foo())
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c.tolist(), a + b)
+                self.check_obj(c)
+
+    def test_iterator1(self):
+        a = bitarray()
+        a.extend(iter([3, 9, 0, 1, -2]))
+        self.assertEqual(a, bitarray('11011'))
+
+        for a in self.randomlists():
+            for b in self.randomlists():
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(iter(b))
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c.tolist(), a + b)
+                self.check_obj(c)
+
+    def test_iterator2(self):
+        try:
+            import itertools
+        except ImportError:
+            return
+        a = bitarray()
+        a.extend(itertools.repeat(True, 23))
+        self.assertEqual(a, bitarray(23 * '1'))
+
+    def test_string01(self):
+        a = bitarray()
+        a.extend('0110111')
+        self.assertEqual(a, bitarray('0110111'))
+
+        for a in self.randomlists():
+            for b in self.randomlists():
+                c = bitarray(a)
+                idc = id(c)
+                c.extend(''.join(['0', '1'][x] for x in b))
+                self.assertEqual(id(c), idc)
+                self.assertEqual(c.tolist(), a + b)
+                self.check_obj(c)
+
+    def test_extend_self(self):
+        a = bitarray()
+        a.extend(a)
+        self.assertEqual(a, bitarray())
+
+        a = bitarray('1')
+        a.extend(a)
+        self.assertEqual(a, bitarray('11'))
+
+        a = bitarray('110')
+        a.extend(a)
+        self.assertEqual(a, bitarray('110110'))
+
+        for a in self.randombitarrays():
+            b = bitarray(a)
+            a.extend(a)
+            self.assertEqual(a, b + b)
+
+
+tests.append(ExtendTests)
+
+# ---------------------------------------------------------------------------
+
+class MethodTests(unittest.TestCase, Util):
+
+    def test_append(self):
+        a = bitarray()
+        a.append(True)
+        a.append(False)
+        a.append(False)
+        self.assertEQUAL(a, bitarray('100'))
+
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            b = a
+            b.append(1)
+            self.assert_(a is b)
+            self.check_obj(b)
+            self.assertEQUAL(b, bitarray(aa+[1], endian=a.endian()))
+            b.append('')
+            self.assertEQUAL(b, bitarray(aa+[1, 0], endian=a.endian()))
+
+
+    def test_insert(self):
+        a = bitarray()
+        b = a
+        a.insert(0, True)
+        self.assert_(a is b)
+        self.assertEqual(a, bitarray('1'))
+        self.assertRaises(TypeError, a.insert)
+        self.assertRaises(TypeError, a.insert, None)
+
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            for _ in range(50):
+                item = bool(randint(0, 1))
+                pos = randint(-len(a) - 2, len(a) + 2)
+                a.insert(pos, item)
+                aa.insert(pos, item)
+                self.assertEqual(a.tolist(), aa)
+                self.check_obj(a)
+
+
+    def test_index(self):
+        a = bitarray()
+        for i in (True, False, 1, 0):
+            self.assertRaises(ValueError, a.index, i)
+
+        a = bitarray(100 * [False])
+        self.assertRaises(ValueError, a.index, True)
+        a[20] = a[27] = 1
+        self.assertEqual(a.index(42), 20)
+        self.assertEqual(a.index(0), 0)
+
+        a = bitarray(200 * [True])
+        self.assertRaises(ValueError, a.index, False)
+        a[173] = a[187] = 0
+        self.assertEqual(a.index(False), 173)
+        self.assertEqual(a.index(True), 0)
+
+        for n in range(50):
+            for m in range(n):
+                a = bitarray(n)
+                a.setall(0)
+                self.assertRaises(ValueError, a.index, 1)
+                a[m] = 1
+                self.assertEqual(a.index(1), m)
+
+                a.setall(1)
+                self.assertRaises(ValueError, a.index, 0)
+                a[m] = 0
+                self.assertEqual(a.index(0), m)
+
+    def test_index2(self):
+        a = bitarray('00001000' '00000000' '0010000')
+        self.assertEqual(a.index(1), 4)
+        self.assertEqual(a.index(1, 1), 4)
+        self.assertEqual(a.index(0, 4), 5)
+        self.assertEqual(a.index(1, 5), 18)
+        self.assertRaises(ValueError, a.index, 1, 5, 18)
+        self.assertRaises(ValueError, a.index, 1, 19)
+
+    def test_index3(self):
+        a = bitarray(2000)
+        a.setall(0)
+        for _ in range(3):
+            a[randint(0, 1999)] = 1
+        aa = a.tolist()
+        for _ in range(100):
+            start = randint(0, 2000)
+            stop = randint(0, 2000)
+            try:
+                res1 = a.index(1, start, stop)
+            except ValueError:
+                res1 = None
+            try:
+                res2 = aa.index(1, start, stop)
+            except ValueError:
+                res2 = None
+            self.assertEqual(res1, res2)
+
+
+    def test_count(self):
+        a = bitarray('10011')
+        self.assertEqual(a.count(), 3)
+        self.assertEqual(a.count(True), 3)
+        self.assertEqual(a.count(False), 2)
+        self.assertEqual(a.count(1), 3)
+        self.assertEqual(a.count(0), 2)
+        self.assertRaises(TypeError, a.count, 'A')
+
+        for i in range(0, 256):
+            a = bitarray()
+            a.frombytes(to_bytes(chr(i)))
+            self.assertEqual(a.count(), a.to01().count('1'))
+
+        for a in self.randombitarrays():
+            self.assertEqual(a.count(), a.count(1))
+            self.assertEqual(a.count(1), a.to01().count('1'))
+            self.assertEqual(a.count(0), a.to01().count('0'))
+
+
+    def test_search(self):
+        a = bitarray('')
+        self.assertEqual(a.search(bitarray('0')), [])
+        self.assertEqual(a.search(bitarray('1')), [])
+
+        a = bitarray('1')
+        self.assertEqual(a.search(bitarray('0')), [])
+        self.assertEqual(a.search(bitarray('1')), [0])
+        self.assertEqual(a.search(bitarray('11')), [])
+
+        a = bitarray(100*'1')
+        self.assertEqual(a.search(bitarray('0')), [])
+        self.assertEqual(a.search(bitarray('1')), list(range(100)))
+
+        a = bitarray('10010101110011111001011')
+        for limit in range(10):
+            self.assertEqual(a.search(bitarray('011'), limit),
+                             [6, 11, 20][:limit])
+        self.assertRaises(ValueError, a.search, bitarray())
+        self.assertRaises(TypeError, a.search, '010')
+
+    def test_itersearch(self):
+        a = bitarray('10011')
+        self.assertRaises(ValueError, a.itersearch, bitarray())
+        self.assertRaises(TypeError, a.itersearch, '')
+        it = a.itersearch(bitarray('1'))
+        self.assertEqual(next(it), 0)
+        self.assertEqual(next(it), 3)
+        self.assertEqual(next(it), 4)
+        self.assertStopIteration(it)
+
+    def test_search2(self):
+        a = bitarray('10011')
+        for s, res in [('0',     [1, 2]),  ('1', [0, 3, 4]),
+                       ('01',    [2]),     ('11', [3]),
+                       ('000',   []),      ('1001', [0]),
+                       ('011',   [2]),     ('0011', [1]),
+                       ('10011', [0]),     ('100111', [])]:
+            b = bitarray(s)
+            self.assertEqual(a.search(b), res)
+            self.assertEqual([p for p in a.itersearch(b)], res)
+
+    def test_search3(self):
+        a = bitarray('10010101110011111001011')
+        for s, res in [('011', [6, 11, 20]),
+                       ('111', [7, 12, 13, 14]), # note the overlap
+                       ('1011', [5, 19]),
+                       ('100', [0, 9, 16])]:
+            b = bitarray(s)
+            self.assertEqual(a.search(b), res)
+            self.assertEqual(list(a.itersearch(b)), res)
+            self.assertEqual([p for p in a.itersearch(b)], res)
+
+
+    def test_fill(self):
+        a = bitarray('')
+        self.assertEqual(a.fill(), 0)
+        self.assertEqual(len(a), 0)
+
+        a = bitarray('101')
+        self.assertEqual(a.fill(), 5)
+        self.assertEQUAL(a, bitarray('10100000'))
+        self.assertEqual(a.fill(), 0)
+        self.assertEQUAL(a, bitarray('10100000'))
+
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            la = len(a)
+            b = a
+            self.assert_(0 <= b.fill() < 8)
+            self.assertEqual(b.endian(), a.endian())
+            bb = b.tolist()
+            lb = len(b)
+            self.assert_(a is b)
+            self.check_obj(b)
+            if la % 8 == 0:
+                self.assertEqual(bb, aa)
+                self.assertEqual(lb, la)
+            else:
+                self.assert_(lb % 8 == 0)
+                self.assertNotEqual(bb, aa)
+                self.assertEqual(bb[:la], aa)
+                self.assertEqual(b[la:], (lb-la)*bitarray('0'))
+                self.assert_(0 < lb-la < 8)
+
+
+    def test_sort(self):
+        a = bitarray('1101000')
+        a.sort()
+        self.assertEqual(a, bitarray('0000111'))
+
+        a = bitarray('1101000')
+        a.sort(reverse=True)
+        self.assertEqual(a, bitarray('1110000'))
+        a.sort(reverse=False)
+        self.assertEqual(a, bitarray('0000111'))
+        a.sort(True)
+        self.assertEqual(a, bitarray('1110000'))
+        a.sort(False)
+        self.assertEqual(a, bitarray('0000111'))
+
+        self.assertRaises(TypeError, a.sort, 'A')
+
+        for a in self.randombitarrays():
+            ida = id(a)
+            rev = randint(0, 1)
+            a.sort(rev)
+            self.assertEqual(a, bitarray(sorted(a.tolist(), reverse=rev)))
+            self.assertEqual(id(a), ida)
+
+
+    def test_reverse(self):
+        self.assertRaises(TypeError, bitarray().reverse, 42)
+
+        for x, y in [('', ''), ('1', '1'), ('10', '01'), ('001', '100'),
+                     ('1110', '0111'), ('11100', '00111'),
+                     ('011000', '000110'), ('1101100', '0011011'),
+                     ('11110000', '00001111'),
+                     ('11111000011', '11000011111'),
+                     ('11011111' '00100000' '000111',
+                      '111000' '00000100' '11111011')]:
+            a = bitarray(x)
+            a.reverse()
+            self.assertEQUAL(a, bitarray(y))
+
+        for a in self.randombitarrays():
+            aa = a.tolist()
+            ida = id(a)
+            a.reverse()
+            self.assertEqual(ida, id(a))
+            self.assertEQUAL(a, bitarray(aa[::-1], endian=a.endian()))
+
+
+    def test_tolist(self):
+        a = bitarray()
+        self.assertEqual(a.tolist(), [])
+
+        a = bitarray('110')
+        self.assertEqual(a.tolist(), [True, True, False])
+
+        for lst in self.randomlists():
+            a = bitarray(lst)
+            self.assertEqual(a.tolist(), lst)
+
+
+    def test_remove(self):
+        a = bitarray()
+        for i in (True, False, 1, 0):
+            self.assertRaises(ValueError, a.remove, i)
+
+        a = bitarray(21)
+        a.setall(0)
+        self.assertRaises(ValueError, a.remove, 1)
+        a.setall(1)
+        self.assertRaises(ValueError, a.remove, 0)
+
+        a = bitarray('1010110')
+        for val, res in [(False, '110110'), (True, '10110'),
+                         (1, '0110'), (1, '010'), (0, '10'),
+                         (0, '1'), (1, '')]:
+            a.remove(val)
+            self.assertEQUAL(a, bitarray(res))
+
+        a = bitarray('0010011')
+        b = a
+        b.remove('1')
+        self.assert_(b is a)
+        self.assertEQUAL(b, bitarray('000011'))
+
+
+    def test_pop(self):
+        for x, n, r, y in [('1', 0, True, ''),
+                           ('0', -1, False, ''),
+                           ('0011100', 3, True, '001100')]:
+            a = bitarray(x)
+            self.assertEqual(a.pop(n), r)
+            self.assertEqual(a, bitarray(y))
+
+        a = bitarray('01')
+        self.assertEqual(a.pop(), True)
+        self.assertEqual(a.pop(), False)
+        self.assertRaises(IndexError, a.pop)
+
+        for a in self.randombitarrays():
+            self.assertRaises(IndexError, a.pop, len(a))
+            self.assertRaises(IndexError, a.pop, -len(a)-1)
+            if len(a) == 0:
+                continue
+            aa = a.tolist()
+            enda = a.endian()
+            self.assertEqual(a.pop(), aa[-1])
+            self.check_obj(a)
+            self.assertEqual(a.endian(), enda)
+
+        for a in self.randombitarrays():
+            if len(a) == 0:
+                continue
+            n = randint(-len(a), len(a)-1)
+            aa = a.tolist()
+            self.assertEqual(a.pop(n), aa[n])
+            aa.pop(n)
+            self.assertEqual(a, bitarray(aa))
+            self.check_obj(a)
+
+
+    def test_setall(self):
+        a = bitarray(5)
+        a.setall(True)
+        self.assertEQUAL(a, bitarray('11111'))
+
+        for a in self.randombitarrays():
+            val = randint(0, 1)
+            b = a
+            b.setall(val)
+            self.assertEqual(b, bitarray(len(b) * [val]))
+            self.assert_(a is b)
+            self.check_obj(b)
+
+
+    def test_bytereverse(self):
+        for x, y in [('', ''),
+                     ('1', '0'),
+                     ('1011', '0000'),
+                     ('111011', '001101'),
+                     ('11101101', '10110111'),
+                     ('000000011', '100000000'),
+                     ('11011111' '00100000' '000111',
+                      '11111011' '00000100' '001110')]:
+            a = bitarray(x)
+            a.bytereverse()
+            self.assertEqual(a, bitarray(y))
+
+        for i in range(256):
+            a = bitarray()
+            a.frombytes(to_bytes(chr(i)))
+            aa = a.tolist()
+            b = a
+            b.bytereverse()
+            self.assertEqual(b, bitarray(aa[::-1]))
+            self.assert_(a is b)
+            self.check_obj(b)
+
+
+tests.append(MethodTests)
+
+# ---------------------------------------------------------------------------
+
+class StringTests(unittest.TestCase, Util):
+
+    def randombytes(self):
+        for n in range(1, 20):
+            yield to_bytes(''.join(chr(randint(0, 255))
+                                   for x in range(n)))
+
+    def test_frombytes(self):
+        a = bitarray(endian='big')
+        a.frombytes(to_bytes('A'))
+        self.assertEqual(a, bitarray('01000001'))
+
+        b = a
+        b.frombytes(to_bytes('BC'))
+        self.assertEQUAL(b, bitarray('01000001' '01000010' '01000011',
+                                     endian='big'))
+        self.assert_(b is a)
+
+        for b in self.randombitarrays():
+            c = b.__copy__()
+            b.frombytes(to_bytes(''))
+            self.assertEQUAL(b, c)
+
+        for b in self.randombitarrays():
+            for s in self.randombytes():
+                a = bitarray(endian=b.endian())
+                a.frombytes(s)
+                c = b.__copy__()
+                b.frombytes(s)
+                self.assertEQUAL(b[-len(a):], a)
+                self.assertEQUAL(b[:-len(a)], c)
+                self.assertEQUAL(c + a, b)
+
+
+    def test_tobytes(self):
+        a = bitarray()
+        self.assertEqual(a.tobytes(), to_bytes(''))
+
+        for end in ('big', 'little'):
+            a = bitarray(endian=end)
+            a.frombytes(to_bytes('foo'))
+            self.assertEqual(a.tobytes(), to_bytes('foo'))
+
+            for s in self.randombytes():
+                a = bitarray(endian=end)
+                a.frombytes(s)
+                self.assertEqual(a.tobytes(), s)
+
+        for n, s in [(1, '\x01'), (2, '\x03'), (3, '\x07'), (4, '\x0f'),
+                     (5, '\x1f'), (6, '\x3f'), (7, '\x7f'), (8, '\xff'),
+                     (12, '\xff\x0f'), (15, '\xff\x7f'), (16, '\xff\xff'),
+                     (17, '\xff\xff\x01'), (24, '\xff\xff\xff')]:
+            a = bitarray(n, endian='little')
+            a.setall(1)
+            self.assertEqual(a.tobytes(), to_bytes(s))
+
+
+    def test_unpack(self):
+        a = bitarray('01')
+        self.assertEqual(a.unpack(), to_bytes('\x00\xff'))
+        self.assertEqual(a.unpack(to_bytes('A')), to_bytes('A\xff'))
+        self.assertEqual(a.unpack(to_bytes('0'), to_bytes('1')),
+                         to_bytes('01'))
+        self.assertEqual(a.unpack(one=to_bytes('\x01')),
+                         to_bytes('\x00\x01'))
+        self.assertEqual(a.unpack(zero=to_bytes('A')),
+                         to_bytes('A\xff'))
+        self.assertEqual(a.unpack(one=to_bytes('t'), zero=to_bytes('f')),
+                         to_bytes('ft'))
+
+        self.assertRaises(TypeError, a.unpack,
+                          to_bytes('a'), zero=to_bytes('b'))
+        self.assertRaises(TypeError, a.unpack, foo=to_bytes('b'))
+
+        for a in self.randombitarrays():
+            self.assertEqual(a.unpack(to_bytes('0'), to_bytes('1')),
+                             to_bytes(a.to01()))
+
+            b = bitarray()
+            b.pack(a.unpack())
+            self.assertEqual(b, a)
+
+            b = bitarray()
+            b.pack(a.unpack(to_bytes('\x01'), to_bytes('\x00')))
+            b.invert()
+            self.assertEqual(b, a)
+
+
+    def test_pack(self):
+        a = bitarray()
+        a.pack(to_bytes('\x00'))
+        self.assertEqual(a, bitarray('0'))
+        a.pack(to_bytes('\xff'))
+        self.assertEqual(a, bitarray('01'))
+        a.pack(to_bytes('\x01\x00\x7a'))
+        self.assertEqual(a, bitarray('01101'))
+
+        a = bitarray()
+        for n in range(256):
+            a.pack(to_bytes(chr(n)))
+        self.assertEqual(a, bitarray('0' + 255 * '1'))
+
+        self.assertRaises(TypeError, a.pack, 0)
+        if is_py3k:
+            self.assertRaises(TypeError, a.pack, '1')
+        self.assertRaises(TypeError, a.pack, [1, 3])
+        self.assertRaises(TypeError, a.pack, bitarray())
+
+
+tests.append(StringTests)
+
+# ---------------------------------------------------------------------------
+
+class FileTests(unittest.TestCase, Util):
+
+    def setUp(self):
+        self.tmpdir = tempfile.mkdtemp()
+        self.tmpfname = os.path.join(self.tmpdir, 'testfile')
+
+    def tearDown(self):
+        shutil.rmtree(self.tmpdir)
+
+
+    def test_pickle(self):
+        from pickle import load, dump
+
+        for v in range(3):
+            for a in self.randombitarrays():
+                fo = open(self.tmpfname, 'wb')
+                dump(a, fo, v)
+                fo.close()
+                b = load(open(self.tmpfname, 'rb'))
+                self.assert_(b is not a)
+                self.assertEQUAL(a, b)
+
+    def test_cPickle(self):
+        if is_py3k:
+            return
+        from cPickle import load, dump
+
+        for v in range(3):
+            for a in self.randombitarrays():
+                fo = open(self.tmpfname, 'wb')
+                dump(a, fo, v)
+                fo.close()
+                b = load(open(self.tmpfname, 'rb'))
+                self.assert_(b is not a)
+                self.assertEQUAL(a, b)
+
+    def test_shelve(self):
+        if sys.version_info[:2] < (2, 5):
+            return
+        import shelve, hashlib
+
+        d = shelve.open(self.tmpfname)
+        stored = []
+        for a in self.randombitarrays():
+            key = hashlib.md5(repr(a).encode() +
+                              a.endian().encode()).hexdigest()
+            d[key] = a
+            stored.append((key, a))
+        d.close()
+        del d
+
+        d = shelve.open(self.tmpfname)
+        for k, v in stored:
+            self.assertEQUAL(d[k], v)
+        d.close()
+
+
+    def test_fromfile_wrong_args(self):
+        b = bitarray()
+        self.assertRaises(TypeError, b.fromfile)
+        self.assertRaises(TypeError, b.fromfile, StringIO()) # file not open
+        self.assertRaises(TypeError, b.fromfile, 42)
+        self.assertRaises(TypeError, b.fromfile, 'bar')
+
+
+    def test_from_empty_file(self):
+        fo = open(self.tmpfname, 'wb')
+        fo.close()
+
+        a = bitarray()
+        a.fromfile(open(self.tmpfname, 'rb'))
+        self.assertEqual(a, bitarray())
+
+
+    def test_from_large_file(self):
+        N = 100000
+
+        fo = open(self.tmpfname, 'wb')
+        fo.write(N * to_bytes('X'))
+        fo.close()
+
+        a = bitarray()
+        a.fromfile(open(self.tmpfname, 'rb'))
+        self.assertEqual(len(a), 8 * N)
+        self.assertEqual(a.buffer_info()[1], N)
+        # make sure there is no over-allocation
+        self.assertEqual(a.buffer_info()[4], N)
+
+
+    def test_fromfile_Foo(self):
+        fo = open(self.tmpfname, 'wb')
+        fo.write(to_bytes('Foo\n'))
+        fo.close()
+
+        a = bitarray(endian='big')
+        a.fromfile(open(self.tmpfname, 'rb'))
+        self.assertEqual(a, bitarray('01000110011011110110111100001010'))
+
+        a = bitarray(endian='little')
+        a.fromfile(open(self.tmpfname, 'rb'))
+        self.assertEqual(a, bitarray('01100010111101101111011001010000'))
+
+        a = bitarray('1', endian='little')
+        a.fromfile(open(self.tmpfname, 'rb'))
+        self.assertEqual(a, bitarray('101100010111101101111011001010000'))
+
+        for n in range(20):
+            a = bitarray(n, endian='little')
+            a.setall(1)
+            a.fromfile(open(self.tmpfname, 'rb'))
+            self.assertEqual(a,
+                             n*bitarray('1') +
+                             bitarray('01100010111101101111011001010000'))
+
+
+    def test_fromfile_n(self):
+        a = bitarray()
+        a.fromstring('ABCDEFGHIJ')
+        fo = open(self.tmpfname, 'wb')
+        a.tofile(fo)
+        fo.close()
+
+        b = bitarray()
+        f = open(self.tmpfname, 'rb')
+        b.fromfile(f, 1);     self.assertEqual(b.tostring(), 'A')
+        f.read(1)
+        b = bitarray()
+        b.fromfile(f, 2);     self.assertEqual(b.tostring(), 'CD')
+        b.fromfile(f, 1);     self.assertEqual(b.tostring(), 'CDE')
+        b.fromfile(f, 0);     self.assertEqual(b.tostring(), 'CDE')
+        b.fromfile(f);        self.assertEqual(b.tostring(), 'CDEFGHIJ')
+        b.fromfile(f);        self.assertEqual(b.tostring(), 'CDEFGHIJ')
+        f.close()
+
+        b = bitarray()
+        f = open(self.tmpfname, 'rb')
+        f.read(1);
+        self.assertRaises(EOFError, b.fromfile, f, 10)
+        f.close()
+        self.assertEqual(b.tostring(), 'BCDEFGHIJ')
+
+        b = bitarray()
+        f = open(self.tmpfname, 'rb')
+        b.fromfile(f);
+        self.assertEqual(b.tostring(), 'ABCDEFGHIJ')
+        self.assertRaises(EOFError, b.fromfile, f, 1)
+        f.close()
+
+
+    def test_tofile(self):
+        a = bitarray()
+        f = open(self.tmpfname, 'wb')
+        a.tofile(f)
+        f.close()
+
+        fi = open(self.tmpfname, 'rb')
+        self.assertEqual(fi.read(), to_bytes(''))
+        fi.close()
+
+        a = bitarray('01000110011011110110111100001010', endian='big')
+        f = open(self.tmpfname, 'wb')
+        a.tofile(f)
+        f.close()
+
+        fi = open(self.tmpfname, 'rb')
+        self.assertEqual(fi.read(), to_bytes('Foo\n'))
+        fi.close()
+
+        for a in self.randombitarrays():
+            b = bitarray(a, endian='big')
+            fo = open(self.tmpfname, 'wb')
+            b.tofile(fo)
+            fo.close()
+
+            s = open(self.tmpfname, 'rb').read()
+            self.assertEqual(len(s), a.buffer_info()[1])
+
+        for n in range(3):
+            a.fromstring(n * 'A')
+            self.assertRaises(TypeError, a.tofile)
+            self.assertRaises(TypeError, a.tofile, StringIO())
+
+            f = open(self.tmpfname, 'wb')
+            a.tofile(f)
+            f.close()
+            self.assertRaises(TypeError, a.tofile, f)
+
+        for n in range(20):
+            a = n * bitarray('1', endian='little')
+            fo = open(self.tmpfname, 'wb')
+            a.tofile(fo)
+            fo.close()
+
+            s = open(self.tmpfname, 'rb').read()
+            self.assertEqual(len(s), a.buffer_info()[1])
+
+            b = a.__copy__()
+            b.fill()
+
+            c = bitarray(endian='little')
+            c.frombytes(s)
+            self.assertEqual(c, b)
+
+
+tests.append(FileTests)
+
+# ---------------------------------------------------------------------------
+
+class PrefixCodeTests(unittest.TestCase, Util):
+
+    def test_encode_errors(self):
+        a = bitarray()
+        self.assertRaises(TypeError, a.encode, 0, '')
+        self.assertRaises(ValueError, a.encode, {}, '')
+        self.assertRaises(TypeError, a.encode, {'a':42}, '')
+        self.assertRaises(ValueError, a.encode, {'a': bitarray()}, '')
+        # 42 not iterable
+        self.assertRaises(TypeError, a.encode, {'a': bitarray('0')}, 42)
+        self.assertEqual(len(a), 0)
+
+    def test_encode_string(self):
+        a = bitarray()
+        d = {'a': bitarray('0')}
+        a.encode(d, '')
+        self.assertEqual(a, bitarray())
+        a.encode(d, 'a')
+        self.assertEqual(a, bitarray('0'))
+        self.assertEqual(d, {'a': bitarray('0')})
+
+    def test_encode_list(self):
+        a = bitarray()
+        d = {'a':bitarray('0')}
+        a.encode(d, [])
+        self.assertEqual(a, bitarray())
+        a.encode(d, ['a'])
+        self.assertEqual(a, bitarray('0'))
+        self.assertEqual(d, {'a': bitarray('0')})
+
+    def test_encode_iter(self):
+        a = bitarray()
+        d = {'a': bitarray('0'), 'b': bitarray('1')}
+        a.encode(d, iter('abba'))
+        self.assertEqual(a, bitarray('0110'))
+
+        def foo():
+            for c in 'bbaabb':
+                yield c
+
+        a.encode(d, foo())
+        self.assertEqual(a, bitarray('0110110011'))
+        self.assertEqual(d, {'a': bitarray('0'), 'b': bitarray('1')})
+
+    def test_encode(self):
+        d = {'I': bitarray('1'),
+             'l': bitarray('01'),
+             'a': bitarray('001'),
+             'n': bitarray('000')}
+        a = bitarray()
+        a.encode(d, 'Ilan')
+        self.assertEqual(a, bitarray('101001000'))
+        a.encode(d, 'a')
+        self.assertEqual(a, bitarray('101001000001'))
+        self.assertEqual(d, {'I': bitarray('1'), 'l': bitarray('01'),
+                             'a': bitarray('001'), 'n': bitarray('000')})
+        self.assertRaises(ValueError, a.encode, d, 'arvin')
+
+
+    def test_decode_check_codedict(self):
+        a = bitarray()
+        self.assertRaises(TypeError, a.decode, 0)
+        self.assertRaises(ValueError, a.decode, {})
+        # 42 not iterable
+        self.assertRaises(TypeError, a.decode, {'a':42})
+        self.assertRaises(ValueError, a.decode, {'a':bitarray()})
+
+    def test_decode_simple(self):
+        d = {'I': bitarray('1'),
+             'l': bitarray('01'),
+             'a': bitarray('001'),
+             'n': bitarray('000')}
+        dcopy = dict(d)
+        a = bitarray('101001000')
+        self.assertEqual(a.decode(d), ['I', 'l', 'a', 'n'])
+        self.assertEqual(d, dcopy)
+        self.assertEqual(a, bitarray('101001000'))
+
+    def test_iterdecode_simple(self):
+        d = {'I': bitarray('1'),
+             'l': bitarray('01'),
+             'a': bitarray('001'),
+             'n': bitarray('000')}
+        dcopy = dict(d)
+        a = bitarray('101001000')
+        self.assertEqual(list(a.iterdecode(d)), ['I', 'l', 'a', 'n'])
+        self.assertEqual(d, dcopy)
+        self.assertEqual(a, bitarray('101001000'))
+
+    def test_decode_empty(self):
+        d = {'a': bitarray('1')}
+        a = bitarray()
+        self.assertEqual(a.decode(d), [])
+        self.assertEqual(d, {'a': bitarray('1')})
+        # test decode iterator
+        self.assertEqual(list(a.iterdecode(d)), [])
+        self.assertEqual(d, {'a': bitarray('1')})
+        self.assertEqual(len(a), 0)
+
+    def test_decode_no_term(self):
+        d = {'a': bitarray('0'), 'b': bitarray('111')}
+        a = bitarray('011')
+        self.assertRaises(ValueError, a.decode, d)
+        self.assertEqual(a, bitarray('011'))
+        self.assertEqual(d, {'a': bitarray('0'), 'b': bitarray('111')})
+
+    def test_decode_buggybitarray(self):
+        d = {'a': bitarray('0')}
+        a = bitarray('1')
+        self.assertRaises(ValueError, a.decode, d)
+        self.assertEqual(a, bitarray('1'))
+        self.assertEqual(d, {'a': bitarray('0')})
+
+    def test_iterdecode_no_term(self):
+        d = {'a': bitarray('0'), 'b': bitarray('111')}
+        a = bitarray('011')
+        it = a.iterdecode(d)
+        if not is_py3k:
+            self.assertEqual(it.next(), 'a')
+            self.assertRaises(ValueError, it.next)
+        self.assertEqual(a, bitarray('011'))
+        self.assertEqual(d, {'a': bitarray('0'), 'b': bitarray('111')})
+
+    def test_iterdecode_buggybitarray(self):
+        d = {'a': bitarray('0')}
+        a = bitarray('1')
+        it = a.iterdecode(d)
+        if not is_py3k:
+            self.assertRaises(ValueError, it.next)
+        self.assertEqual(a, bitarray('1'))
+        self.assertEqual(d, {'a': bitarray('0')})
+
+    def test_decode_buggybitarray2(self):
+        d = {'a': bitarray('00'), 'b': bitarray('01')}
+        a = bitarray('1')
+        self.assertRaises(ValueError, a.decode, d)
+        self.assertEqual(a, bitarray('1'))
+
+    def test_iterdecode_buggybitarray2(self):
+        d = {'a': bitarray('00'), 'b': bitarray('01')}
+        a = bitarray('1')
+        it = a.iterdecode(d)
+        if not is_py3k:
+            self.assertRaises(ValueError, it.next)
+        self.assertEqual(a, bitarray('1'))
+
+    def test_decode_ambiguous_code(self):
+        d = {'a': bitarray('0'), 'b': bitarray('0'), 'c': bitarray('1')}
+        a = bitarray()
+        self.assertRaises(ValueError, a.decode, d)
+        self.assertRaises(ValueError, a.iterdecode, d)
+
+    def test_decode_ambiguous2(self):
+        d = {'a': bitarray('01'), 'b': bitarray('01'), 'c': bitarray('1')}
+        a = bitarray()
+        self.assertRaises(ValueError, a.decode, d)
+        self.assertRaises(ValueError, a.iterdecode, d)
+
+    def test_miscitems(self):
+        d = {None : bitarray('00'),
+             0    : bitarray('110'),
+             1    : bitarray('111'),
+             ''   : bitarray('010'),
+             2    : bitarray('011')}
+        a = bitarray()
+        a.encode(d, [None, 0, 1, '', 2])
+        self.assertEqual(a, bitarray('00110111010011'))
+        self.assertEqual(a.decode(d), [None, 0, 1, '', 2])
+        # iterator
+        it = a.iterdecode(d)
+        self.assertEqual(next(it), None)
+        self.assertEqual(next(it), 0)
+        self.assertEqual(next(it), 1)
+        self.assertEqual(next(it), '')
+        self.assertEqual(next(it), 2)
+        self.assertStopIteration(it)
+
+    def test_real_example(self):
+        code = {' '  : bitarray('001'),
+                '.'  : bitarray('0101010'),
+                'a'  : bitarray('0110'),
+                'b'  : bitarray('0001100'),
+                'c'  : bitarray('000011'),
+                'd'  : bitarray('01011'),
+                'e'  : bitarray('111'),
+                'f'  : bitarray('010100'),
+                'g'  : bitarray('101000'),
+                'h'  : bitarray('00000'),
+                'i'  : bitarray('1011'),
+                'j'  : bitarray('0111101111'),
+                'k'  : bitarray('00011010'),
+                'l'  : bitarray('01110'),
+                'm'  : bitarray('000111'),
+                'n'  : bitarray('1001'),
+                'o'  : bitarray('1000'),
+                'p'  : bitarray('101001'),
+                'q'  : bitarray('00001001101'),
+                'r'  : bitarray('1101'),
+                's'  : bitarray('1100'),
+                't'  : bitarray('0100'),
+                'u'  : bitarray('000100'),
+                'v'  : bitarray('0111100'),
+                'w'  : bitarray('011111'),
+                'x'  : bitarray('0000100011'),
+                'y'  : bitarray('101010'),
+                'z'  : bitarray('00011011110')}
+        a = bitarray()
+        message = 'the quick brown fox jumps over the lazy dog.'
+        a.encode(code, message)
+        self.assertEqual(a, bitarray('01000000011100100001001101000100101100'
+          '00110001101000100011001101100001111110010010101001000000010001100'
+          '10111101111000100000111101001110000110000111100111110100101000000'
+          '0111001011100110000110111101010100010101110001010000101010'))
+        self.assertEqual(''.join(a.decode(code)), message)
+        self.assertEqual(''.join(a.iterdecode(code)), message)
+
+
+tests.append(PrefixCodeTests)
+
+# -------------- Buffer Interface (Python 2.7 only for now) ----------------
+
+class BufferInterfaceTests(unittest.TestCase):
+
+    def test_read1(self):
+        a = bitarray('01000001' '01000010' '01000011', endian='big')
+        v = memoryview(a)
+        self.assertEqual(len(v), 3)
+        self.assertEqual(v[0], 65 if is_py3k else 'A')
+        self.assertEqual(v[:].tobytes(), b'ABC')
+        a[13] = 1
+        self.assertEqual(v[:].tobytes(), b'AFC')
+
+    def test_read2(self):
+        a = bitarray([randint(0, 1) for d in range(8000)])
+        v = memoryview(a)
+        self.assertEqual(len(v), 1000)
+        b = a[345 * 8 : 657 * 8]
+        self.assertEqual(v[345:657].tobytes(), b.tobytes())
+        self.assertEqual(v[:].tobytes(), a.tobytes())
+
+    def test_write(self):
+        a = bitarray(800000)
+        a.setall(0)
+        v = memoryview(a)
+        self.assertFalse(v.readonly)
+        v[50000] = 255 if is_py3k else '\xff'
+        self.assertEqual(a[399999:400009], bitarray('0111111110'))
+        a[400003] = 0
+        self.assertEqual(a[399999:400009], bitarray('0111011110'))
+        v[30001:30004] = b'ABC'
+        self.assertEqual(a[240000:240040].tobytes(), b'\x00ABC\x00')
+
+if sys.version_info[:2] >= (2, 7):
+    tests.append(BufferInterfaceTests)
+
+# ---------------------------------------------------------------------------
+
+def run(verbosity=1, repeat=1):
+    print('bitarray is installed in: %s' % os.path.dirname(__file__))
+    print('bitarray version: %s' % __version__)
+    print('Python version: %s' % sys.version)
+
+    suite = unittest.TestSuite()
+    for cls in tests:
+        for _ in range(repeat):
+            suite.addTest(unittest.makeSuite(cls))
+
+    runner = unittest.TextTestRunner(verbosity=verbosity)
+    return runner.run(suite)
+
+
+if __name__ == '__main__':
+    run()
diff --git a/shell/ext-py/bitarray-0.9.0/setup.py b/shell/ext-py/bitarray-0.9.0/setup.py
new file mode 100644
index 0000000..0473b1f
--- /dev/null
+++ b/shell/ext-py/bitarray-0.9.0/setup.py
@@ -0,0 +1,47 @@
+import re
+from os.path import join
+from distutils.core import setup, Extension
+
+
+kwds = {}
+try:
+    kwds['long_description'] = open('README.rst').read()
+except IOError:
+    pass
+
+# Read version from bitarray/__init__.py
+pat = re.compile(r'__version__\s*=\s*(\S+)', re.M)
+data = open(join('bitarray', '__init__.py')).read()
+kwds['version'] = eval(pat.search(data).group(1))
+
+
+setup(
+    name = "bitarray",
+    author = "Ilan Schnell",
+    author_email = "ilanschnell@gmail.com",
+    url = "https://github.com/ilanschnell/bitarray",
+    license = "PSF",
+    classifiers = [
+        "License :: OSI Approved :: Python Software Foundation License",
+        "Development Status :: 5 - Production/Stable",
+        "Intended Audience :: Developers",
+        "Operating System :: OS Independent",
+        "Programming Language :: C",
+        "Programming Language :: Python :: 2",
+        "Programming Language :: Python :: 2.5",
+        "Programming Language :: Python :: 2.6",
+        "Programming Language :: Python :: 2.7",
+        "Programming Language :: Python :: 3",
+        "Programming Language :: Python :: 3.3",
+        "Programming Language :: Python :: 3.4",
+        "Programming Language :: Python :: 3.5",
+        "Programming Language :: Python :: 3.6",
+        "Programming Language :: Python :: 3.7",
+        "Topic :: Utilities",
+    ],
+    description = "efficient arrays of booleans -- C extension",
+    packages = ["bitarray"],
+    ext_modules = [Extension(name = "bitarray._bitarray",
+                             sources = ["bitarray/_bitarray.c"])],
+    **kwds
+)
diff --git a/shell/impala_client.py b/shell/impala_client.py
index b1c49fb..9141b11 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -17,6 +17,9 @@
 # specific language governing permissions and limitations
 # under the License.
 
+from bitarray import bitarray
+import operator
+import re
 import sasl
 import sys
 import time
@@ -24,19 +27,74 @@ import time
 from beeswaxd import BeeswaxService
 from beeswaxd.BeeswaxService import QueryState
 from ExecStats.ttypes import TExecStats
-from ImpalaService import ImpalaService
+from ImpalaService import ImpalaService, ImpalaHiveServer2Service
+from ImpalaService.ImpalaHiveServer2Service import (TGetRuntimeProfileReq,
+    TGetExecSummaryReq, TPingImpalaHS2ServiceReq, TCloseImpalaOperationReq)
 from ErrorCodes.ttypes import TErrorCode
 from Status.ttypes import TStatus
+from TCLIService.TCLIService import (TExecuteStatementReq, TOpenSessionReq,
+    TCloseSessionReq, TProtocolVersion, TStatusCode, TGetOperationStatusReq,
+    TOperationState, TFetchResultsReq, TFetchOrientation, TGetLogReq,
+    TGetResultSetMetadataReq, TTypeId, TCancelOperationReq)
 from thrift.protocol import TBinaryProtocol
 from thrift_sasl import TSaslClientTransport
 from thrift.transport.TSocket import TSocket
 from thrift.transport.TTransport import TBufferedTransport, TTransportException
 from thrift.Thrift import TApplicationException, TException
 
-class RpcStatus:
-  """Convenience enum to describe Rpc return statuses"""
-  OK = 0
-  ERROR = 1
+
+# Helpers to extract and convert HS2's representation of values to the display version.
+# An entry must be added to this map for each supported type. HS2's TColumn has many
+# different typed field, each of which has a 'values' and a 'nulls' field. The first
+# element of each tuple is a "getter" function that extracts the appropriate member from
+# TColumn for the given TTypeId. The second element is a "stringifier" function that
+# converts a single value to its display representation. If the value is already a
+# string and does not need conversion for display, the stringifier can be None.
+HS2_VALUE_CONVERTERS = {
+    TTypeId.BOOLEAN_TYPE: (operator.attrgetter('boolVal'),
+     lambda b: 'true' if b else 'false'),
+    TTypeId.TINYINT_TYPE: (operator.attrgetter('byteVal'), str),
+    TTypeId.SMALLINT_TYPE: (operator.attrgetter('i16Val'), str),
+    TTypeId.INT_TYPE: (operator.attrgetter('i32Val'), str),
+    TTypeId.BIGINT_TYPE: (operator.attrgetter('i64Val'), str),
+    TTypeId.TIMESTAMP_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.FLOAT_TYPE: (operator.attrgetter('doubleVal'), str),
+    TTypeId.DOUBLE_TYPE: (operator.attrgetter('doubleVal'), str),
+    TTypeId.STRING_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.DECIMAL_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.BINARY_TYPE: (operator.attrgetter('binaryVal'), str),
+    TTypeId.VARCHAR_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.CHAR_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.MAP_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.ARRAY_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.STRUCT_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.UNION_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.NULL_TYPE: (operator.attrgetter('stringVal'), None),
+    TTypeId.DATE_TYPE: (operator.attrgetter('stringVal'), None)
+}
+
+# Regular expression that matches the progress line added to HS2 logs by
+# the Impala server.
+HS2_LOG_PROGRESS_REGEX = re.compile("Query.*Complete \([0-9]* out of [0-9]*\)\n")
+
+
+class QueryOptionLevels:
+  """These are the levels used when displaying query options.
+  The values correspond to the ones in TQueryOptionLevel"""
+  REGULAR = 0
+  ADVANCED = 1
+  DEVELOPMENT = 2
+  DEPRECATED = 3
+  REMOVED = 4
+
+  # Map from level name to the level's numeric value.
+  NAME_TO_VALUES = {'REGULAR': 0, 'ADVANCED': 1, 'DEVELOPMENT': 2,
+                    'DEPRECATED': 3, 'REMOVED': 4}
+
+  @classmethod
+  def from_string(cls, string):
+    """Return the integral value based on the string. Defaults to DEVELOPMENT."""
+    return cls.NAME_TO_VALUES.get(string.upper(), cls.DEVELOPMENT)
 
 class RPCException(Exception):
     def __init__(self, value=""):
@@ -59,11 +117,22 @@ class DisconnectedException(Exception):
 class QueryCancelledByShellException(Exception): pass
 
 
+class MissingThriftMethodException(Exception):
+  """Thrown if a Thrift method that the client tried to call is missing."""
+  def __init__(self, value=""):
+      self.value = value
+
+  def __str__(self):
+      return self.value
+
+
 def print_to_stderr(message):
   print >> sys.stderr, message
 
 class ImpalaClient(object):
-
+  """Base class for shared functionality between HS2 and Beeswax. Includes stub methods
+  for methods that are expected to be implemented in the subclasses.
+  TODO: when beeswax support is removed, merge this with ImpalaHS2Client."""
   def __init__(self, impalad, kerberos_host_fqdn, use_kerberos=False,
                kerberos_service_name="impala", use_ssl=False, ca_cert=None, user=None,
                ldap_password=None, use_ldap=False, client_connect_timeout_ms=60000,
@@ -83,7 +152,6 @@ class ImpalaClient(object):
     self.client_connect_timeout_ms = int(client_connect_timeout_ms)
     self.default_query_options = {}
     self.query_option_levels = {}
-    self.query_state = QueryState._NAMES_TO_VALUES
     self.fetch_batch_size = 1024
     # This is set from ImpalaShell's signal handler when a query is cancelled
     # from command line via CTRL+C. It is used to suppress error messages of
@@ -91,30 +159,258 @@ class ImpalaClient(object):
     self.is_query_cancelled = False
     self.verbose = verbose
 
-  def _options_to_string_list(self, set_query_options):
-    return ["%s=%s" % (k, v) for (k, v) in set_query_options.iteritems()]
+  def connect(self):
+    """Creates a connection to an Impalad instance. Returns a tuple with the impala
+    version string and the webserver address, otherwise raises an exception. If the client
+    was already connected, closes the previous connection."""
+    self.close_connection()
 
-  def build_default_query_options_dict(self):
-    """The default query options are retrieved from a rpc call, and are dependent
-    on the impalad to which a connection has been established. They need to be
-    refreshed each time a connection is made. This is particularly helpful when
-    there is a version mismatch between the shell and the impalad.
-    """
+    sock, self.transport = self._get_socket_and_transport()
+    if self.client_connect_timeout_ms > 0:
+      sock.setTimeout(self.client_connect_timeout_ms)
+    self.transport.open()
+    if self.verbose:
+      print_to_stderr('Opened TCP connection to %s:%s' % (self.impalad_host,
+          self.impalad_port))
+    # Setting a timeout of None disables timeouts on sockets
+    sock.setTimeout(None)
+    protocol = TBinaryProtocol.TBinaryProtocolAccelerated(self.transport)
+    self.imp_service = self._get_thrift_client(protocol)
+    self.connected = True
     try:
-      get_default_query_options = self.imp_service.get_default_configuration(False)
+      self._open_session()
+      return self._ping_impala_service()
     except:
-      return
-    rpc_result = self._do_rpc(lambda: get_default_query_options)
-    options, status = rpc_result
-    if status != RpcStatus.OK:
-      raise RPCException("Unable to retrieve default query options")
-    for option in options:
-      self.default_query_options[option.key.upper()] = option.value
-      # If connected to an Impala that predates IMPALA-2181 then the received options
-      # wouldn't contain a level attribute. In this case the query_option_levels
-      # map is left empty.
-      if option.level is not None:
-        self.query_option_levels[option.key.upper()] = option.level
+      # Ensure we are in a disconnected state if we failed above.
+      self.close_connection()
+      raise
+
+  def _get_thrift_client(self, protocol):
+    """Instantiate a thrift client with the provided protocol."""
+    raise NotImplementedError()
+
+  def _open_session(self):
+    """Does any work to open a session for a new connection.
+    Also sets up self.default_query_options and self.query_option_levels
+    to include metadata about the options supported by the server."""
+    raise NotImplementedError()
+
+  def is_connected(self):
+    """Returns True if the current Impala connection is alive and False otherwise."""
+    if not self.connected:
+      return False
+    try:
+      self._ping_impala_service()
+      return True
+      # Catch exceptions that are associated with communication errors.
+    except TException:
+      self.close_connection()
+      return False
+    except RPCException:
+      self.close_connection()
+      return False
+    except DisconnectedException:
+      self.close_connection()
+      return False
+
+  def close_connection(self):
+    """Close any open sessions and close the connection if still open."""
+    raise NotImplementedError()
+
+  def _close_transport(self):
+    """Closes transport if not closed and set self.connected to False. This is the last
+    step of close_connection()."""
+    if self.transport:
+      self.transport.close()
+    self.connected = False
+
+  def _ping_impala_service(self):
+    """Pings the Impala service to ensure it can receive RPCs. Returns a tuple with
+    the impala version string and the webserver address. Raise TException, RPCException,
+    DisconnectedException or MissingThriftMethodException if it cannot successfully
+    communicate with the Impala daemon."""
+    raise NotImplementedError()
+
+  def execute_query(self, query_str, set_query_options):
+    """Execute the query 'query_str' asynchronously on the server with options dictionary
+    'set_query_options' and return a query handle that can be used for subsequent
+    ImpalaClient method calls for the query. The handle returned is
+    implementation-dependent but is guaranteed to have an 'is_closed' member
+    that reflects whether the query was closed with close_query() or close_dml()"""
+    raise NotImplementedError()
+
+  def get_query_id_str(self, last_query_handle):
+    """Return the standard string representation of an Impala query ID, e.g.
+    'd74d8ce632c9d4d0:75c5a51100000000'"""
+    raise NotImplementedError()
+
+  def wait_to_finish(self, last_query_handle, periodic_callback=None):
+    """Wait until the results can be fetched for 'last_query_handle' or until the
+    query encounters an error or is cancelled. Raises an exception if the query
+    encounters an error or is cancelled or if we lose connection to the impalad.
+    If 'periodic_callback' is provided, it is called periodically with no arguments."""
+    loop_start = time.time()
+    while True:
+      query_state = self.get_query_state(last_query_handle)
+      if query_state == self.FINISHED_STATE:
+        break
+      elif query_state in (self.ERROR_STATE, self.CANCELED_STATE):
+        if self.connected:
+          # TODO: does this do the right thing for a cancelled query?
+          raise QueryStateException(self.get_error_log(last_query_handle))
+        else:
+          raise DisconnectedException("Not connected to impalad.")
+
+      if periodic_callback is not None: periodic_callback()
+      time.sleep(self._get_sleep_interval(loop_start))
+
+  def get_query_state(self, last_query_handle):
+    """Return the query state string for 'last_query_handle'. Returns self.ERROR_STATE
+    if there is an error communicating with the server or the client is disconnected.
+    """
+    raise NotImplementedError()
+
+  def get_column_names(self, last_query_handle):
+    """Get a list of column names for the query. The query must have a result set."""
+    raise NotImplementedError()
+
+  def expect_result_metadata(self, query_str, query_handle):
+    """Given a query string and handle, return True if impalad expects result metadata."""
+    raise NotImplementedError()
+
+  def fetch(self, query_handle):
+    """Returns an iterable of batches of result rows. Each batch is an iterable of rows.
+    Each row is an iterable of strings in the format in which they should be displayed
+    Tries to ensure that the batches have a granularity of self.fetch_batch_size but
+    does not guarantee it.
+    """
+    result_rows = None
+    for rows in self._fetch_one_batch(query_handle):
+      if result_rows:
+        result_rows.extend(rows)
+      else:
+        result_rows = rows
+      if len(result_rows) > self.fetch_batch_size:
+        yield result_rows
+        result_rows = None
+
+    # Return the final batch of rows.
+    if result_rows:
+      yield result_rows
+
+  def _fetch_one_batch(self, query_handle):
+    """Returns an iterable of batches of result rows up to self.fetch_batch_size. Does
+    not need to consolidate those batches into larger batches."""
+    raise NotImplementedError()
+
+  # TODO: when we remove Beeswax, we could merge close_dml() and close_query()
+  # because the CloseImpalaOperation() response contains enough information to
+  # differentiate between DML and non-DML.
+  def close_dml(self, last_query_handle):
+    """Fetches the results of a DML query. Returns a tuple containing the
+       number of rows modified and the number of row errors, in that order. If the DML
+       operation doesn't return 'num_row_errors', then the second element in the tuple
+       is None. Returns None if the query was not closed successfully. Not idempotent."""
+    raise NotImplementedError()
+
+  def close_query(self, last_query_handle):
+    """Close the query handle. Idempotent - after the first attempt, closing the same
+    query handle is a no-op. Returns True if the query was closed
+    successfully or False otherwise."""
+    raise NotImplementedError()
+
+  def cancel_query(self, last_query_handle):
+    """Cancel a query on a keyboard interrupt from the shell. Return True if the
+    query was previously cancelled or if the cancel operation succeeded. Return
+    False otherwise."""
+    raise NotImplementedError()
+
+  def get_runtime_profile(self, last_query_handle):
+    """Get the runtime profile string from the server. Returns None if
+    an error was encountered."""
+    raise NotImplementedError()
+
+  def get_summary(self, last_query_handle):
+    """Get the thrift TExecSummary from the server. Returns None if
+    an error was encountered."""
+    raise NotImplementedError()
+
+  def _get_warn_or_error_log(self, last_query_handle, warn):
+    """Returns all messages from the error log prepended with 'WARNINGS:' or 'ERROR:' for
+    last_query_handle, depending on whether warn is True or False. Note that the error
+    log may contain messages that are not errors (e.g. warnings)."""
+    raise NotImplementedError()
+
+  def get_warning_log(self, last_query_handle):
+    """Returns all messages from the error log prepended with 'WARNINGS:' for
+    last_query_handle. Note that the error log may contain messages that are not errors
+    (e.g. warnings)."""
+    return self._get_warn_or_error_log(last_query_handle, True)
+
+  def get_error_log(self, last_query_handle):
+    """Returns all messages from the error log prepended with 'ERROR:' for
+    last_query_handle. Note that the error log may contain messages that are not errors
+    (e.g. warnings)."""
+    return self._get_warn_or_error_log(last_query_handle, False)
+
+  def _get_socket_and_transport(self):
+    """Create a Transport.
+
+       A non-kerberized impalad just needs a simple buffered transport. For
+       the kerberized version, a sasl transport is created.
+
+       If SSL is enabled, a TSSLSocket underlies the transport stack; otherwise a TSocket
+       is used.
+       This function returns the socket and the transport object.
+    """
+    if self.use_ssl:
+      # TSSLSocket needs the ssl module, which may not be standard on all Operating
+      # Systems. Only attempt to import TSSLSocket if the user wants an SSL connection.
+      from TSSLSocketWithWildcardSAN import TSSLSocketWithWildcardSAN
+
+    # sasl does not accept unicode strings, explicitly encode the string into ascii.
+    # The kerberos_host_fqdn option exposes the SASL client's hostname attribute to
+    # the user. impala-shell checks to ensure this host matches the host in the kerberos
+    # principal. So when a load balancer is configured to be used, its hostname is
+    # expected by impala-shell. Setting this option to the load balancer hostname allows
+    # impala-shell to connect directly to an impalad.
+    if self.kerberos_host_fqdn is not None:
+      sasl_host = self.kerberos_host_fqdn.split(':')[0].encode('ascii', 'ignore')
+    else:
+      sasl_host = self.impalad_host
+
+    # Always use the hostname and port passed in to -i / --impalad as the host for the
+    # purpose of creating the actual socket.
+    sock_host = self.impalad_host
+    sock_port = self.impalad_port
+    if self.use_ssl:
+      if self.ca_cert is None:
+        # No CA cert means don't try to verify the certificate
+        sock = TSSLSocketWithWildcardSAN(sock_host, sock_port, validate=False)
+      else:
+        sock = TSSLSocketWithWildcardSAN(
+            sock_host, sock_port, validate=True, ca_certs=self.ca_cert)
+    else:
+      sock = TSocket(sock_host, sock_port)
+    if not (self.use_ldap or self.use_kerberos):
+      return sock, TBufferedTransport(sock)
+
+    # Initializes a sasl client
+    def sasl_factory():
+      sasl_client = sasl.Client()
+      sasl_client.setAttr("host", sasl_host)
+      if self.use_ldap:
+        sasl_client.setAttr("username", self.user)
+        sasl_client.setAttr("password", self.ldap_password)
+      else:
+        sasl_client.setAttr("service", self.kerberos_service_name)
+      sasl_client.init()
+      return sasl_client
+    # GSSASPI is the underlying mechanism used by kerberos to authenticate.
+    if self.use_kerberos:
+      return sock, TSaslClientTransport(sasl_factory, "GSSAPI", sock)
+    else:
+      return sock, TSaslClientTransport(sasl_factory, "PLAIN", sock)
 
   def build_summary_table(self, summary, idx, is_fragment_root, indent_level,
       new_indent_level, output):
@@ -238,228 +534,468 @@ class ImpalaClient(object):
       output += first_child_output
     return idx
 
-  def is_connected(self):
-    """Returns True if the current Impala connection is alive and False otherwise."""
-    if self.connected:
+  def _get_sleep_interval(self, start_time):
+    """Returns a step function of time to sleep in seconds before polling
+    again. Maximum sleep is 1s, minimum is 0.1s"""
+    elapsed = time.time() - start_time
+    if elapsed < 10.0:
+      return 0.1
+    elif elapsed < 60.0:
+      return 0.5
+    return 1.0
+
+  def _check_connected(self):
+    """Raise DiconnectedException if the client is not connected."""
+    if not self.connected:
+      raise DisconnectedException("Not connected (use CONNECT to establish a connection)")
+
+
+class ImpalaHS2Client(ImpalaClient):
+  """Impala client. Uses the HS2 protocol plus Impala-specific extensions."""
+  def __init__(self, *args, **kwargs):
+    super(ImpalaHS2Client, self).__init__(*args, **kwargs)
+    self.FINISHED_STATE = TOperationState._NAMES_TO_VALUES["FINISHED_STATE"]
+    self.ERROR_STATE = TOperationState._NAMES_TO_VALUES["ERROR_STATE"]
+    self.CANCELED_STATE = TOperationState._NAMES_TO_VALUES["CANCELED_STATE"]
+
+    # If connected, this is the handle returned by the OpenSession RPC that needs
+    # to be passed into most HS2 RPCs.
+    self.session_handle = None
+
+  def _get_thrift_client(self, protocol):
+    return ImpalaHiveServer2Service.Client(protocol)
+
+  def _open_session(self):
+    open_session_req = TOpenSessionReq(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6,
+        username=self.user)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.OpenSession(open_session_req))
+    self._check_hs2_rpc_status(resp.status)
+    assert (resp.serverProtocolVersion ==
+            TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6), resp.serverProtocolVersion
+    # TODO: ensure it's closed if needed
+    self.session_handle = resp.sessionHandle
+
+    # List all of the query options and their levels.
+    set_all_handle = self.execute_query("set all", {})
+    self.default_query_options = {}
+    self.query_option_levels = {}
+    for rows in self.fetch(set_all_handle):
+      for name, value, level in rows:
+        self.default_query_options[name.upper()] = value
+        self.query_option_levels[name.upper()] = QueryOptionLevels.from_string(level)
+    try:
+      self.close_query(set_all_handle)
+    except Exception, e:
+      print str(e), type(e)
+      raise
+
+  def close_connection(self):
+    if self.session_handle is not None:
+      # Attempt to close session explicitly. Do not fail if there is an error
+      # doing so. We still need to close the transport and we can rely on the
+      # server to clean up the session.
       try:
-        return self.imp_service.PingImpalaService()
-      except TException:
-        return False
-    else:
-      return False
+        req = TCloseSessionReq(self.session_handle)
+        resp = self._do_hs2_rpc(lambda: self.imp_service.CloseSession(req))
+        self._check_hs2_rpc_status(resp.status)
+      except Exception, e:
+        print("Warning: close session RPC failed: {0}, {1}".format(str(e), type(e)))
+      self.session_handle = None
+    self._close_transport()
+
+  def _ping_impala_service(self):
+    req = TPingImpalaHS2ServiceReq(self.session_handle)
+    try:
+      resp = self.imp_service.PingImpalaHS2Service(req)
+    except TApplicationException, t:
+      if t.type == TApplicationException.UNKNOWN_METHOD:
+        raise MissingThriftMethodException(t.message)
+      raise
+    self._check_hs2_rpc_status(resp.status)
+    return (resp.version, resp.webserver_address)
+
+  def _create_query_req(self, query_str, set_query_options):
+    query = TExecuteStatementReq(sessionHandle=self.session_handle, statement=query_str,
+        confOverlay=set_query_options, runAsync=True)
+    return query
 
-  def connect(self):
-    """Creates a connection to an Impalad instance
+  def execute_query(self, query_str, set_query_options):
+    """Execute the query 'query_str' asynchronously on the server with options dictionary
+    'set_query_options' and return a query handle that can be used for subsequent
+    ImpalaClient method calls for the query."""
+    query = self._create_query_req(query_str, set_query_options)
+    self.is_query_cancelled = False
+    resp = self._do_hs2_rpc(lambda: self.imp_service.ExecuteStatement(query))
+    if resp.status.statusCode != TStatusCode.SUCCESS_STATUS:
+      raise QueryStateException("ERROR: {0}".format(resp.status.errorMessage))
+    handle = resp.operationHandle
+    if handle.hasResultSet:
+      req = TGetResultSetMetadataReq(handle)
+      resp = self._do_hs2_rpc(lambda: self.imp_service.GetResultSetMetadata(req))
+      self._check_hs2_rpc_status(resp.status)
+      assert resp.schema is not None, resp
+      # Attach the schema to the handle for convenience.
+      handle.schema = resp.schema
+    handle.is_closed = False
+    return handle
+
+  def get_query_id_str(self, last_query_handle):
+    # The binary representation is present in the query handle but we need to
+    # massage it into the expected string representation. C++ and Java code
+    # treats the low and high half as two 64-bit little-endian integers and
+    # as a result prints the hex representation in the reverse order to how
+    # bytes are laid out in guid.
+    guid_bytes = last_query_handle.operationId.guid
+    return "{0}:{1}".format(guid_bytes[7::-1].encode('hex_codec'),
+                            guid_bytes[16:7:-1].encode('hex_codec'))
+
+  def _fetch_one_batch(self, query_handle):
+    assert query_handle.hasResultSet
+    prim_types = [column.typeDesc.types[0].primitiveEntry.type
+                  for column in query_handle.schema.columns]
+    col_value_converters = [HS2_VALUE_CONVERTERS[prim_type]
+                        for prim_type in prim_types]
+    while True:
+      req = TFetchResultsReq(query_handle, TFetchOrientation.FETCH_NEXT,
+          self.fetch_batch_size)
+      resp = self._do_hs2_rpc(lambda: self.imp_service.FetchResults(req))
+      self._check_hs2_rpc_status(resp.status)
+
+      # Transpose the columns into a row-based format for more convenient processing
+      # for the display code. This is somewhat inefficient, but performance is comparable
+      # to the old Beeswax code.
+      yield self._transpose(col_value_converters, resp.results.columns)
+      if not resp.hasMoreRows:
+        return
+
+  def _transpose(self, col_value_converters, columns):
+    """Transpose the columns from a TFetchResultsResp into the row format returned
+    by fetch() with all the values converted into their string representations for
+    display. Uses the getters and stringifiers provided in col_value_converters[i]
+    for column i."""
+    tcols = [col_value_converters[i][0](col) for i, col in enumerate(columns)]
+    num_rows = len(tcols[0].values)
+    # Preallocate rows for efficiency.
+    rows = [[None] * len(tcols) for i in xrange(num_rows)]
+    for col_idx, tcol in enumerate(tcols):
+      is_null = bitarray(endian='little')
+      is_null.frombytes(tcol.nulls)
+      stringifier = col_value_converters[col_idx][1]
+      # Skip stringification if not needed. This makes large extracts of tpch.orders
+      # ~8% faster according to benchmarks.
+      if stringifier is None:
+        for row_idx, row in enumerate(rows):
+          row[col_idx] = 'NULL' if is_null[row_idx] else tcol.values[row_idx]
+      else:
+        for row_idx, row in enumerate(rows):
+          row[col_idx] = 'NULL' if is_null[row_idx] else stringifier(tcol.values[row_idx])
+    return rows
 
-    The instance of the impala service is then pinged to
-    test the connection and get back the server version
-    """
-    if self.transport is not None:
-      self.transport.close()
-      self.transport = None
+  def close_dml(self, last_query_handle):
+    req = TCloseImpalaOperationReq(last_query_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.CloseImpalaOperation(req))
+    self._check_hs2_rpc_status(resp.status)
+    if not resp.dml_result:
+      raise RPCException("Impala DML operation did not return DML statistics.")
+
+    num_rows = sum([int(k) for k in resp.dml_result.rows_modified.values()])
+    last_query_handle.is_closed = True
+    return (num_rows, resp.dml_result.num_row_errors)
+
+  def close_query(self, last_query_handle):
+    # Set a member in the handle to make sure that it is idempotent
+    if last_query_handle.is_closed:
+      return True
+    req = TCloseImpalaOperationReq(last_query_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.CloseImpalaOperation(req))
+    last_query_handle.is_closed = True
+    return self._is_hs2_nonerror_status(resp.status.statusCode)
 
-    self.connected = False
-    sock, self.transport = self._get_socket_and_transport()
-    if self.client_connect_timeout_ms > 0:
-      sock.setTimeout(self.client_connect_timeout_ms)
-    self.transport.open()
-    if self.verbose:
-      print_to_stderr('Opened TCP connection to %s:%s' % (self.impalad_host,
-          self.impalad_port))
-    # Setting a timeout of None disables timeouts on sockets
-    sock.setTimeout(None)
-    protocol = TBinaryProtocol.TBinaryProtocol(self.transport)
-    self.imp_service = ImpalaService.Client(protocol)
-    result = self.ping_impala_service()
-    self.connected = True
-    return result
+  def cancel_query(self, last_query_handle):
+    # Cancel sets query_state to ERROR_STATE before calling cancel() in the
+    # co-ordinator, so we don't need to wait.
+    if last_query_handle.is_closed:
+      return True
+    req = TCancelOperationReq(last_query_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.CancelOperation(req),
+        suppress_error_on_cancel=False)
+    return self._is_hs2_nonerror_status(resp.status.statusCode)
 
-  def ping_impala_service(self):
-    return self.imp_service.PingImpalaService()
+  def get_query_state(self, last_query_handle):
+    resp = self._do_hs2_rpc(
+        lambda: self.imp_service.GetOperationStatus(
+          TGetOperationStatusReq(last_query_handle)))
+    self._check_hs2_rpc_status(resp.status)
+    return resp.operationState
 
-  def close_connection(self):
-    """Close the transport if it's still open"""
-    if self.transport:
-      self.transport.close()
+  def get_runtime_profile(self, last_query_handle):
+    req = TGetRuntimeProfileReq(last_query_handle, self.session_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.GetRuntimeProfile(req))
+    self._check_hs2_rpc_status(resp.status)
+    return resp.profile
 
-  def _get_socket_and_transport(self):
-    """Create a Transport.
+  def get_summary(self, last_query_handle):
+    req = TGetExecSummaryReq(last_query_handle, self.session_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.GetExecSummary(req))
+    self._check_hs2_rpc_status(resp.status)
+    return resp.summary
 
-       A non-kerberized impalad just needs a simple buffered transport. For
-       the kerberized version, a sasl transport is created.
+  def get_column_names(self, last_query_handle):
+    # The handle has the schema embedded in it.
+    assert last_query_handle.hasResultSet
+    return [column.columnName for column in last_query_handle.schema.columns]
 
-       If SSL is enabled, a TSSLSocket underlies the transport stack; otherwise a TSocket
-       is used.
-       This function returns the socket and the transport object.
-    """
-    if self.use_ssl:
-      # TSSLSocket needs the ssl module, which may not be standard on all Operating
-      # Systems. Only attempt to import TSSLSocket if the user wants an SSL connection.
-      from TSSLSocketWithWildcardSAN import TSSLSocketWithWildcardSAN
+  def expect_result_metadata(self, query_str, query_handle):
+    """ Given a query string, return True if impalad expects result metadata."""
+    return query_handle.hasResultSet
 
-    # sasl does not accept unicode strings, explicitly encode the string into ascii.
-    # The kerberos_host_fqdn option exposes the SASL client's hostname attribute to
-    # the user. impala-shell checks to ensure this host matches the host in the kerberos
-    # principal. So when a load balancer is configured to be used, its hostname is expected by
-    # impala-shell. Setting this option to the load balancer hostname allows impala-shell to
-    # connect directly to an impalad.
-    if self.kerberos_host_fqdn is not None:
-      sasl_host = self.kerberos_host_fqdn.split(':')[0].encode('ascii', 'ignore')
-    else:
-      sasl_host = self.impalad_host
+  def _get_warn_or_error_log(self, last_query_handle, warn):
+    """Returns all messages from the error log prepended with 'WARNINGS:' or 'ERROR:' for
+    last_query_handle, depending on whether warn is True or False. Note that the error
+    log may contain messages that are not errors (e.g. warnings)."""
+    if last_query_handle is None:
+      return "Query could not be executed"
+    req = TGetLogReq(last_query_handle)
+    resp = self._do_hs2_rpc(lambda: self.imp_service.GetLog(req))
+    self._check_hs2_rpc_status(resp.status)
 
-    # Always use the hostname and port passed in to -i / --impalad as the host for the purpose of
-    # creating the actual socket.
-    sock_host = self.impalad_host
-    sock_port = self.impalad_port
-    if self.use_ssl:
-      if self.ca_cert is None:
-        # No CA cert means don't try to verify the certificate
-        sock = TSSLSocketWithWildcardSAN(sock_host, sock_port, validate=False)
-      else:
-        sock = TSSLSocketWithWildcardSAN(sock_host, sock_port, validate=True, ca_certs=self.ca_cert)
-    else:
-      sock = TSocket(sock_host, sock_port)
-    if not (self.use_ldap or self.use_kerberos):
-      return sock, TBufferedTransport(sock)
+    log = resp.log
+    # Strip progress message out of HS2 log.
+    log = HS2_LOG_PROGRESS_REGEX.sub("", log)
+    if log and log.strip():
+      type_str = "WARNINGS" if warn is True else "ERROR"
+      return "%s: %s" % (type_str, log)
+    return ""
 
-    # Initializes a sasl client
-    def sasl_factory():
-      sasl_client = sasl.Client()
-      sasl_client.setAttr("host", sasl_host)
-      if self.use_ldap:
-        sasl_client.setAttr("username", self.user)
-        sasl_client.setAttr("password", self.ldap_password)
-      else:
-        sasl_client.setAttr("service", self.kerberos_service_name)
-      sasl_client.init()
-      return sasl_client
-    # GSSASPI is the underlying mechanism used by kerberos to authenticate.
-    if self.use_kerberos:
-      return sock, TSaslClientTransport(sasl_factory, "GSSAPI", sock)
+  def _do_hs2_rpc(self, rpc, suppress_error_on_cancel=True):
+    """Executes the provided 'rpc' callable and tranlates any exceptions in the
+    appropriate exception for the shell. Exceptions raised include:
+    * DisconnectedException if the client cannot communicate with the server.
+    * QueryCancelledByShellException if 'suppress_error_on_cancel' is true, the RPC
+      fails and the query was cancelled from the shell (i.e. self.is_query_cancelled).
+    * MissingThriftMethodException if the thrift method is not implemented on the server.
+    Does not validate any status embedded in the returned RPC message."""
+    self._check_connected()
+    try:
+      return rpc()
+    except TTransportException, e:
+      # issue with the connection with the impalad
+      raise DisconnectedException("Error communicating with impalad: %s" % e)
+    except TApplicationException, t:
+      # Suppress the errors from cancelling a query that is in waiting_to_finish state
+      if suppress_error_on_cancel and self.is_query_cancelled:
+        raise QueryCancelledByShellException()
+      if t.type == TApplicationException.UNKNOWN_METHOD:
+        raise MissingThriftMethodException(t.message)
+      raise RPCException("Application Exception : {0}".format(t))
+
+  def _check_hs2_rpc_status(self, status):
+    """If the TCLIService.TStatus 'status' is an error status the raise an exception
+    with an appropriate error message. The exceptions raised are:
+    * QueryCancelledByShellException if the RPC fails and the query was cancelled from
+      the shell (i.e. self.is_query_cancelled).
+    * QueryStateException if the query is not registered on the server.
+    * RPCException in all other cases."""
+    if status.statusCode == TStatusCode.ERROR_STATUS:
+      # Suppress the errors from cancelling a query that is in fetch state
+      if self.is_query_cancelled:
+        raise QueryCancelledByShellException()
+      raise RPCException("ERROR: {0}".format(status.errorMessage))
+    elif status.statusCode == TStatusCode.INVALID_HANDLE_STATUS:
+      if self.is_query_cancelled:
+        raise QueryCancelledByShellException()
+      raise QueryStateException('Error: Stale query handle')
     else:
-      return sock, TSaslClientTransport(sasl_factory, "PLAIN", sock)
+      # Treat all non-error codes as success.
+      assert self._is_hs2_nonerror_status(status.statusCode), status.statusCode
+
+  def _is_hs2_nonerror_status(self, status_code):
+    """Return whether 'status_code' is a non-error TStatusCode value."""
+    return status_code in (TStatusCode.SUCCESS_STATUS,
+                           TStatusCode.SUCCESS_WITH_INFO_STATUS,
+                           TStatusCode.STILL_EXECUTING_STATUS)
+
+
+class RpcStatus:
+  """Convenience enum used in ImpalaBeeswaxClient to describe Rpc return statuses"""
+  OK = 0
+  ERROR = 1
+
+
+class ImpalaBeeswaxClient(ImpalaClient):
+  """Legacy Beeswax client. Uses the Beeswax protocol plus Impala-specific extensions.
+  TODO: remove once we've phased out beeswax."""
+  def __init__(self, *args, **kwargs):
+    super(ImpalaBeeswaxClient, self).__init__(*args, **kwargs)
+    self.FINISHED_STATE = QueryState._NAMES_TO_VALUES["FINISHED"]
+    self.ERROR_STATE = QueryState._NAMES_TO_VALUES["EXCEPTION"]
+    self.CANCELED_STATE = QueryState._NAMES_TO_VALUES["EXCEPTION"]
+
+  def _get_thrift_client(self, protocol):
+    return ImpalaService.Client(protocol)
+
+  def _options_to_string_list(self, set_query_options):
+    return ["%s=%s" % (k, v) for (k, v) in set_query_options.iteritems()]
+
+  def _open_session(self):
+    # Beeswax doesn't have a "session" concept independent of connections, so
+    # we do not need to explicitly open a sesion. We still need to set up the
+    # query options.
+    #
+    # The default query options are retrieved from a rpc call, and are dependent
+    # on the impalad to which a connection has been established. They need to be
+    # refreshed each time a connection is made. This is particularly helpful when
+    # there is a version mismatch between the shell and the impalad.
+    try:
+      get_default_query_options = self.imp_service.get_default_configuration(False)
+    except Exception:
+      return
+    rpc_result = self._do_beeswax_rpc(lambda: get_default_query_options)
+    options, status = rpc_result
+    if status != RpcStatus.OK:
+      raise RPCException("Unable to retrieve default query options")
+
+    for option in options:
+      self.default_query_options[option.key.upper()] = option.value
+      # If connected to an Impala that predates IMPALA-2181 then the received options
+      # wouldn't contain a level attribute. In this case the query_option_levels
+      # map is left empty.
+      if option.level is not None:
+        self.query_option_levels[option.key.upper()] = option.level
+
+  def close_connection(self):
+    # Beeswax sessions are scoped to the connection, so we only need to close transport.
+    self._close_transport()
+
+  def _ping_impala_service(self):
+    try:
+      resp = self.imp_service.PingImpalaService()
+    except TApplicationException, t:
+      if t.type == TApplicationException.UNKNOWN_METHOD:
+        raise MissingThriftMethodException(t.message)
+      raise
+    return (resp.version, resp.webserver_address)
 
-  def create_beeswax_query(self, query_str, set_query_options):
-    """Create a beeswax query object from a query string"""
+  def _create_query_req(self, query_str, set_query_options):
     query = BeeswaxService.Query()
     query.hadoop_user = self.user
     query.query = query_str
     query.configuration = self._options_to_string_list(set_query_options)
     return query
 
-  def execute_query(self, query):
+  def execute_query(self, query_str, set_query_options):
+    """Execute the query 'query_str' asynchronously on the server with options dictionary
+    'set_query_options' and return a query handle that can be used for subsequent
+    ImpalaClient method calls for the query."""
+    query = self._create_query_req(query_str, set_query_options)
     self.is_query_cancelled = False
-    rpc_result = self._do_rpc(lambda: self.imp_service.query(query))
-    last_query_handle, status = rpc_result
-    if status != RpcStatus.OK:
+    handle, rpc_status = self._do_beeswax_rpc(lambda: self.imp_service.query(query))
+    if rpc_status != RpcStatus.OK:
       raise RPCException("Error executing the query")
-    return last_query_handle
+    handle.is_closed = False
+    return handle
 
-  def wait_to_finish(self, last_query_handle, periodic_callback=None):
-    loop_start = time.time()
-    while True:
-      query_state = self.get_query_state(last_query_handle)
-      if query_state == self.query_state["FINISHED"]:
-        break
-      elif query_state == self.query_state["EXCEPTION"]:
-        if self.connected:
-          raise QueryStateException(self.get_error_log(last_query_handle))
-        else:
-          raise DisconnectedException("Not connected to impalad.")
+  def get_query_id_str(self, last_query_handle):
+    return last_query_handle.id
 
-      if periodic_callback is not None: periodic_callback()
-      time.sleep(self._get_sleep_interval(loop_start))
+  def get_query_state(self, last_query_handle):
+    state, rpc_status = self._do_beeswax_rpc(
+        lambda: self.imp_service.get_state(last_query_handle))
+    if rpc_status != RpcStatus.OK:
+      return self.ERROR_STATE
+    return state
 
-  def fetch(self, query_handle):
-    """Fetch all the results.
-    This function returns a generator to create an iterable of the result rows.
-    """
-    result_rows = []
+  def _fetch_one_batch(self, query_handle):
     while True:
-      rpc_result = self._do_rpc(
-        lambda: self.imp_service.fetch(query_handle, False,
-                                       self.fetch_batch_size))
-
-      result, status = rpc_result
-
-      if status != RpcStatus.OK:
+      result, rpc_status = self._do_beeswax_rpc(
+         lambda: self.imp_service.fetch(query_handle, False,
+                                        self.fetch_batch_size))
+      if rpc_status != RpcStatus.OK:
         raise RPCException()
-
-      result_rows.extend(result.data)
-
-      if len(result_rows) >= self.fetch_batch_size or not result.has_more:
-        rows = [row.split('\t') for row in result_rows]
-        result_rows = []
-        yield rows
-        if not result.has_more:
-          break
+      yield [row.split('\t') for row in result.data]
+      if not result.has_more:
+        return
 
   def close_dml(self, last_query_handle):
-    """Fetches the results of a DML query. Returns a tuple containing the
-       number of rows modified and the number of row errors, in that order. If the DML
-       operation doesn't return 'num_row_errors', then the second element in the tuple
-       is None."""
-    rpc_result = self._do_rpc(
+    insert_result, rpc_status = self._do_beeswax_rpc(
         lambda: self.imp_service.CloseInsert(last_query_handle))
-    insert_result, status = rpc_result
-
-    if status != RpcStatus.OK:
-      raise RPCException()
-
+    if rpc_status != RpcStatus.OK:
+       raise RPCException()
     num_rows = sum([int(k) for k in insert_result.rows_modified.values()])
+    last_query_handle.is_closed = True
     return (num_rows, insert_result.num_row_errors)
 
-  def close_query(self, last_query_handle, query_handle_closed=False):
-    """Close the query handle"""
-    # Make closing a query handle idempotent
-    if query_handle_closed:
+  def close_query(self, last_query_handle):
+    # Set a member in the handle to make sure that it is idempotent
+    if last_query_handle.is_closed:
       return True
-    rpc_result = self._do_rpc(lambda: self.imp_service.close(last_query_handle))
-    _, status = rpc_result
-    return status == RpcStatus.OK
+    _, rpc_status = self._do_beeswax_rpc(
+        lambda: self.imp_service.close(last_query_handle))
+    last_query_handle.is_closed = True
+    return rpc_status == RpcStatus.OK
 
-  def cancel_query(self, last_query_handle, query_handle_closed=False):
-    """Cancel a query on a keyboard interrupt from the shell."""
-    # Cancel sets query_state to EXCEPTION before calling cancel() in the
+  def cancel_query(self, last_query_handle):
+    # Cancel sets query_state to ERROR_STATE before calling cancel() in the
     # co-ordinator, so we don't need to wait.
-    if query_handle_closed:
+    if last_query_handle.is_closed:
       return True
-    rpc_result = self._do_rpc(lambda: self.imp_service.Cancel(last_query_handle),
-        False)
-    _, status = rpc_result
-    return status == RpcStatus.OK
-
-  def get_query_state(self, last_query_handle):
-    rpc_result = self._do_rpc(
-        lambda: self.imp_service.get_state(last_query_handle))
-    state, status = rpc_result
-    if status != RpcStatus.OK:
-      return self.query_state["EXCEPTION"]
-    return state
+    _, rpc_status = self._do_beeswax_rpc(
+        lambda: self.imp_service.Cancel(last_query_handle), False)
+    return rpc_status == RpcStatus.OK
 
   def get_runtime_profile(self, last_query_handle):
-    rpc_result = self._do_rpc(
+    profile, rpc_status = self._do_beeswax_rpc(
         lambda: self.imp_service.GetRuntimeProfile(last_query_handle))
-    profile, status = rpc_result
-    if status == RpcStatus.OK and profile:
+    if rpc_status == RpcStatus.OK and profile:
       return profile
+    return None
 
   def get_summary(self, last_query_handle):
-    """Calls GetExecSummary() for the last query handle"""
-    rpc_result = self._do_rpc(
+    summary, rpc_status = self._do_beeswax_rpc(
       lambda: self.imp_service.GetExecSummary(last_query_handle))
-    summary, status = rpc_result
-    if status == RpcStatus.OK and summary:
+    if rpc_status == RpcStatus.OK and summary:
       return summary
     return None
 
-  def _do_rpc(self, rpc, suppress_error_on_cancel=True):
-    """Executes the provided callable."""
+  def get_column_names(self, last_query_handle):
+    # Note: the code originally ignored the RPC status. don't mess with it.
+    metadata, _ = self._do_beeswax_rpc(
+        lambda: self.imp_service.get_results_metadata(last_query_handle))
+    if metadata is not None:
+      return [fs.name for fs in metadata.schema.fieldSchemas]
 
-    if not self.connected:
-      raise DisconnectedException("Not connected (use CONNECT to establish a connection)")
-      return None, RpcStatus.ERROR
+  def expect_result_metadata(self, query_str, query_handle):
+    # Beeswax doesn't provide us this metadata; try to guess whether to expect it based
+    # on the query string.
+    excluded_query_types = ['use']
+    if True in set(map(query_str.startswith, excluded_query_types)):
+      return False
+    return True
+
+  def _get_warn_or_error_log(self, last_query_handle, warn):
+    if last_query_handle is None:
+      return "Query could not be executed"
+    log, rpc_status = self._do_beeswax_rpc(
+        lambda: self.imp_service.get_log(last_query_handle.log_context))
+    if rpc_status != RpcStatus.OK:
+      type_str = "warn" if warn is True else "error"
+      return "Failed to get %s log: %s" % (type_str, rpc_status)
+    if log and log.strip():
+      type_str = "WARNINGS" if warn is True else "ERROR"
+      return "%s: %s" % (type_str, log)
+    return ""
+
+  def _do_beeswax_rpc(self, rpc, suppress_error_on_cancel=True):
+    """Executes the provided 'rpc' callable. Raises exceptions for most errors,
+    including:
+    * DisconnectedException if the client cannot communicate with the server.
+    * QueryCancelledByShellException if 'suppress_error_on_cancel' is true, the RPC
+      fails and the query was cancelled from the shell (i.e. self.is_query_cancelled).
+    * RPCException if the operation fails with an error status
+    * QueryStateException if the query is not registered on the server.
+    * MissingThriftMethodException if the thrift method is not implemented on the server.
+    Returns RPCStatus.OK on success or RPCStatus.ERROR for any other errors."""
+    self._check_connected()
     try:
       ret = rpc()
       status = RpcStatus.OK
@@ -469,7 +1005,7 @@ class ImpalaClient(object):
       if ret is not None and isinstance(ret, TStatus):
         if ret.status_code != TErrorCode.OK:
           if ret.error_msgs:
-            raise RPCException ('RPC Error: %s' % '\n'.join(ret.error_msgs))
+            raise RPCException('RPC Error: %s' % '\n'.join(ret.error_msgs))
           status = RpcStatus.ERROR
       return ret, status
     except BeeswaxService.QueryNotFoundException:
@@ -487,62 +1023,10 @@ class ImpalaClient(object):
       # issue with the connection with the impalad
       raise DisconnectedException("Error communicating with impalad: %s" % e)
     except TApplicationException, t:
-      # Suppress the errors from cancelling a query that is in waiting_to_finish
-      # state
+      # Suppress the errors from cancelling a query that is in waiting_to_finish state
       if suppress_error_on_cancel and self.is_query_cancelled:
         raise QueryCancelledByShellException()
+      if t.type == TApplicationException.UNKNOWN_METHOD:
+        raise MissingThriftMethodException(t.message)
       raise RPCException("Application Exception : %s" % t)
-    return None, RpcStatus.ERROR
-
-  def _get_sleep_interval(self, start_time):
-    """Returns a step function of time to sleep in seconds before polling
-    again. Maximum sleep is 1s, minimum is 0.1s"""
-    elapsed = time.time() - start_time
-    if elapsed < 10.0:
-      return 0.1
-    elif elapsed < 60.0:
-      return 0.5
-    return 1.0
-
-  def get_column_names(self, last_query_handle):
-    rpc_result = self._do_rpc(
-        lambda: self.imp_service.get_results_metadata(last_query_handle))
-    metadata, _ = rpc_result
-    if not metadata is None:
-      return [fs.name for fs in metadata.schema.fieldSchemas]
-
-  def expect_result_metadata(self, query_str):
-    """ Given a query string, return True if impalad expects result metadata"""
-    excluded_query_types = ['use']
-    if True in set(map(query_str.startswith, excluded_query_types)):
-      return False
-    return True
 
-  def get_warn_or_error_log(self, last_query_handle, warn):
-    """Returns all messages from the error log prepended with 'WARNINGS:' or 'ERROR:' for
-    last_query_handle, depending on whether warn is True or False. Note that the error
-    log may contain messages that are not errors (e.g. warnings)."""
-    if last_query_handle is None:
-      return "Query could not be executed"
-    rpc_result = self._do_rpc(
-        lambda: self.imp_service.get_log(last_query_handle.log_context))
-    log, status = rpc_result
-    if status != RpcStatus.OK:
-      type_str = "warn" if warn is True else "error"
-      return "Failed to get %s log: %s" % (type_str, status)
-    if log and log.strip():
-      type_str = "WARNINGS" if warn is True else "ERROR"
-      return "%s: %s" % (type_str, log)
-    return ""
-
-  def get_warning_log(self, last_query_handle):
-    """Returns all messages from the error log prepended with 'WARNINGS:' for
-    last_query_handle. Note that the error log may contain messages that are not errors
-    (e.g. warnings)."""
-    return self.get_warn_or_error_log(last_query_handle, True)
-
-  def get_error_log(self, last_query_handle):
-    """Returns all messages from the error log prepended with 'ERROR:' for
-    last_query_handle. Note that the error log may contain messages that are not errors
-    (e.g. warnings)."""
-    return self.get_warn_or_error_log(last_query_handle, False)
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index f36f456..553e787 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -35,15 +35,17 @@ import sys
 import textwrap
 import time
 
-from impala_client import (ImpalaClient, DisconnectedException, QueryStateException,
-                           RPCException, TApplicationException,
-                           QueryCancelledByShellException)
+from impala_client import (ImpalaHS2Client, ImpalaBeeswaxClient, DisconnectedException,
+                           QueryStateException, RPCException,
+                           QueryCancelledByShellException, QueryOptionLevels,
+                           MissingThriftMethodException)
 from impala_shell_config_defaults import impala_shell_defaults
 from option_parser import get_option_parser, get_config_from_file
 from shell_output import DelimitedOutputFormatter, OutputStream, PrettyOutputFormatter
 from shell_output import OverwritingStdErrOutputStream
 from subprocess import call
 
+
 VERSION_FORMAT = "Impala Shell v%(version)s (%(git_hash)s) built on %(build_date)s"
 VERSION_STRING = "impala shell build version not available"
 READLINE_UNAVAILABLE_ERROR = "The readline module was either not found or disabled. " \
@@ -58,6 +60,9 @@ try:
 except Exception:
   pass
 
+DEFAULT_BEESWAX_PORT = 21000
+DEFAULT_HS2_PORT = 21050
+
 def strip_comments(sql):
   """sqlparse default implementation of strip comments has a bad performance when parsing
   very large SQL due to the grouping. This is because the default implementation tries to
@@ -80,6 +85,12 @@ class CmdStatus:
   ABORT = True
   ERROR = False
 
+
+class FatalShellException(Exception):
+  """Thrown if a fatal error occurs that requires terminating the shell. The cause of the
+  error should be logged to stderr before raising this exception."""
+  pass
+
 class ImpalaPrettyTable(prettytable.PrettyTable):
   """Patched version of PrettyTable with different unicode handling - instead of throwing
   exceptions when a character can't be converted to unicode, it is replaced with a
@@ -92,15 +103,6 @@ class ImpalaPrettyTable(prettytable.PrettyTable):
       value = unicode(value, self.encoding, "replace")
     return value
 
-class QueryOptionLevels:
-  """These are the levels used when displaying query options.
-  The values correspond to the ones in TQueryOptionLevel"""
-  REGULAR = 0
-  ADVANCED = 1
-  DEVELOPMENT = 2
-  DEPRECATED = 3
-  REMOVED = 4
-
 class QueryOptionDisplayModes:
   REGULAR_OPTIONS_ONLY = 1
   ALL_OPTIONS = 2
@@ -108,6 +110,10 @@ class QueryOptionDisplayModes:
 class ImpalaShell(object, cmd.Cmd):
   """ Simple Impala Shell.
 
+  Implements the context manager interface to ensure client connections and sessions
+  are cleanly torn down. ImpalaShell instances should be used within a "with" statement
+  to ensure correct teardown.
+
   Basic usage: type connect <host:port> to connect to an impalad
   Then issue queries or other commands. Tab-completion should show the set of
   available commands.
@@ -120,7 +126,6 @@ class ImpalaShell(object, cmd.Cmd):
   UNKNOWN_SERVER_VERSION = "Not Connected"
   PROMPT_FORMAT = "[{host}:{port}] {db}> "
   DISCONNECTED_PROMPT = "[Not connected] > "
-  UNKNOWN_WEBSERVER = "0.0.0.0"
   # Message to display in shell when cancelling a query
   CANCELLATION_MESSAGE = ' Cancelling Query'
   # Number of times to attempt cancellation before giving up.
@@ -174,7 +179,7 @@ class ImpalaShell(object, cmd.Cmd):
     self.verbose = options.verbose
     self.prompt = ImpalaShell.DISCONNECTED_PROMPT
     self.server_version = ImpalaShell.UNKNOWN_SERVER_VERSION
-    self.webserver_address = ImpalaShell.UNKNOWN_WEBSERVER
+    self.webserver_address = None
 
     self.current_db = options.default_db
     self.history_file = os.path.expanduser(options.history_file)
@@ -199,12 +204,13 @@ class ImpalaShell(object, cmd.Cmd):
 
     self._populate_command_list()
 
-    self.imp_client = None;
+    self.imp_client = None
+
+    # Used to pass the original unmodified command into do_*() methods.
     self.orig_cmd = None
 
     # Tracks query handle of the last query executed. Used by the 'profile' command.
     self.last_query_handle = None;
-    self.query_handle_closed = None
 
     self.print_summary = options.print_summary
     self.print_progress = options.print_progress
@@ -239,6 +245,12 @@ class ImpalaShell(object, cmd.Cmd):
     # requests between the handler and the main shell thread.
     signal.signal(signal.SIGINT, self._signal_handler)
 
+  def __enter__(self):
+    return self
+
+  def __exit__(self, type, value, traceback):
+    self.close_connection()
+
   def _populate_command_list(self):
     """Populate a list of commands in the shell.
 
@@ -331,21 +343,13 @@ class ImpalaShell(object, cmd.Cmd):
     for x in self.VALID_SHELL_OPTIONS:
       print "\t%s: %s" % (x, self.__dict__[self.VALID_SHELL_OPTIONS[x][1]])
 
-  def _create_beeswax_query(self, args):
-    """Original command should be stored before running the method. The method is usually
-    used in do_* methods and the command is kept at precmd()."""
-    command = self.orig_cmd
-    self.orig_cmd = None
-    if not command:
-      print_to_stderr("Unexpected error: Failed to execute query due to command "
-                      "is missing")
-      sys.exit(1)
+  def _build_query_string(self, leading_comment, cmd, args):
+    """Called to build a query string based on the parts output by parseline():
+    the leading comment, the command name and the arguments to the command."""
     # In order to deduce the correct cmd, parseline stripped the leading comment.
     # To preserve the original query, the leading comment (if exists) will be
     # prepended when constructing the query sent to the Impala front-end.
-    return self.imp_client.create_beeswax_query("%s%s %s" % (self.leading_comment or "",
-                                                command, args),
-                                                self.set_query_options)
+    return "{0}{1} {2}".format(leading_comment or '', cmd or '', args)
 
   def do_shell(self, args):
     """Run a command on the shell
@@ -517,25 +521,46 @@ class ImpalaShell(object, cmd.Cmd):
     return completed_cmd
 
   def _new_impala_client(self):
-    return ImpalaClient(self.impalad, self.kerberos_host_fqdn, self.use_kerberos,
-                        self.kerberos_service_name, self.use_ssl,
-                        self.ca_cert, self.user, self.ldap_password,
-                        self.use_ldap, self.client_connect_timeout_ms, self.verbose)
+    protocol = options.protocol.lower()
+    if protocol == 'hs2':
+      return ImpalaHS2Client(self.impalad, self.kerberos_host_fqdn, self.use_kerberos,
+                          self.kerberos_service_name, self.use_ssl,
+                          self.ca_cert, self.user, self.ldap_password,
+                          self.use_ldap, self.client_connect_timeout_ms, self.verbose)
+    elif protocol == 'beeswax':
+      return ImpalaBeeswaxClient(self.impalad, self.kerberos_host_fqdn, self.use_kerberos,
+                          self.kerberos_service_name, self.use_ssl,
+                          self.ca_cert, self.user, self.ldap_password,
+                          self.use_ldap, self.client_connect_timeout_ms, self.verbose)
+    else:
+      print_to_stderr("Invalid --protocol value {0}, must be beeswax or hs2.".format(
+                      protocol))
+      raise FatalShellException()
+
+  def close_connection(self):
+    """Closes the current Impala connection."""
+    if self.imp_client:
+      self.imp_client.close_connection()
 
   def _signal_handler(self, signal, frame):
     """Handles query cancellation on a Ctrl+C event"""
-    if self.last_query_handle is None or self.query_handle_closed:
+    if self.last_query_handle is None or self.last_query_handle.is_closed:
       raise KeyboardInterrupt()
     # Create a new connection to the impalad and cancel the query.
+    # TODO: this isn't thread-safe with respect to the main thread executing the
+    # query. This probably contributes to glitchiness when cancelling query in
+    # the shell.
     for cancel_try in xrange(ImpalaShell.CANCELLATION_TRIES):
       try:
         self.imp_client.is_query_cancelled = True
-        self.query_handle_closed = True
         print_to_stderr(ImpalaShell.CANCELLATION_MESSAGE)
         new_imp_client = self._new_impala_client()
         new_imp_client.connect()
-        new_imp_client.cancel_query(self.last_query_handle, False)
-        new_imp_client.close_query(self.last_query_handle)
+        try:
+          new_imp_client.cancel_query(self.last_query_handle)
+          new_imp_client.close_query(self.last_query_handle)
+        finally:
+          new_imp_client.close_connection()
         break
       except Exception, e:
         # Suppress harmless errors.
@@ -590,18 +615,19 @@ class ImpalaShell(object, cmd.Cmd):
       command, arg, line, leading_comment = self.parseline(line)
       if not line:
         return self.emptyline()
-      if command is None:
-        return self.default(line)
+      # orig_cmd and last_leading_comment are passed into do_*() functions
+      # via this side mechanism because the cmd module limits us to passing
+      # in the argument list only.
+      self.orig_cmd = command
+      self.last_leading_comment = leading_comment
       self.lastcmd = line
-      if line == 'EOF' :
-        self.lastcmd = ''
-      if command == '':
+      if not command:
         return self.default(line)
+      elif line == 'EOF':
+        self.lastcmd = ''
       else:
         try:
           func = getattr(self, 'do_' + command.lower())
-          self.orig_cmd = command
-          self.leading_comment = leading_comment
         except AttributeError:
           return self.default(line)
         return func(arg)
@@ -612,6 +638,10 @@ class ImpalaShell(object, cmd.Cmd):
     return status
 
   def do_summary(self, args):
+    if not self.last_query_handle:
+      print_to_stderr("Could not retrieve summary: no previous query.")
+      return CmdStatus.ERROR
+
     summary = None
     try:
       summary = self.imp_client.get_summary(self.last_query_handle)
@@ -736,9 +766,9 @@ class ImpalaShell(object, cmd.Cmd):
 
   def do_connect(self, args):
     """Connect to an Impalad instance:
-    Usage: connect, defaults to the fqdn of the localhost and port 21000
+    Usage: connect, defaults to the fqdn of the localhost and the protocol's default port
            connect <hostname:port>
-           connect <hostname>, defaults to port 21000
+           connect <hostname>, defaults to the protocol's default port
 
     """
     # Assume the user wants to connect to the local impalad if no connection string is
@@ -755,9 +785,13 @@ class ImpalaShell(object, cmd.Cmd):
                       "<hostname[:port]>")
       return CmdStatus.ERROR
     elif len(host_port) == 1:
-      host_port.append('21000')
+      if options.protocol.lower() == 'hs2':
+        host_port.append(str(DEFAULT_HS2_PORT))
+      else:
+        assert options.protocol.lower() == 'beeswax'
+        host_port.append(str(DEFAULT_BEESWAX_PORT))
     self.impalad = tuple(host_port)
-    if self.imp_client: self.imp_client.close_connection()
+    self.close_connection()
     self.imp_client = self._new_impala_client()
     self._connect()
     # If the connection fails and the Kerberos has not been enabled,
@@ -781,10 +815,6 @@ class ImpalaShell(object, cmd.Cmd):
       self._print_if_verbose('Server version: %s' % self.server_version)
       self.set_prompt(ImpalaShell.DEFAULT_DB)
       self._validate_database()
-    try:
-      self.imp_client.build_default_query_options_dict()
-    except RPCException, e:
-      print_to_stderr(e)
     # In the case that we lost connection while a command was being entered,
     # we may have a dangling command, clear partial_cmd
     self.partial_cmd = str()
@@ -811,30 +841,42 @@ class ImpalaShell(object, cmd.Cmd):
 
   def _connect(self):
     try:
-      result = self.imp_client.connect()
-      self.server_version = result.version
-      self.webserver_address = result.webserver_address
-    except TApplicationException:
+      self.server_version, self.webserver_address = self.imp_client.connect()
+    except MissingThriftMethodException, e:
+      if options.protocol.lower() == 'beeswax':
+        port_flag = "-beeswax_port"
+        addtl_suggestion = ""
+      else:
+        assert options.protocol.lower() == 'hs2'
+        port_flag = "-hs2_port"
+        addtl_suggestion = (" Also check that the Impala daemon connected to has version "
+                            "3.3 or greater. impala-shell only supports connecting to "
+                            "the HS2 interface of Impala version 3.3 or greater. For "
+                            "older Impala versions you must use the (deprecated) beeswax "
+                            "protocol with the --protocol=beeswax.")
       # We get a TApplicationException if the transport is valid,
       # but the RPC does not exist.
       print_to_stderr("\n".join(textwrap.wrap(
-        "Error: Unable to communicate with impalad service. This "
-        "service may not be an impalad instance. A common problem is "
-        "that the port specified does not match the -beeswax_port flag on "
-        "the underlying impalad. Check host:port and try again.")))
-      self.imp_client.close_connection()
+        "Error: Unable to communicate with impalad service because of the error "
+        "reported below. The service does not implement a required Thrift method. "
+        "The service may not be an Impala Daemon or you may have specified the wrong "
+        "port to connect to. Check host:port to ensure that the port matches the "
+        "{port_flag} flag on the Impala Daemon and try again.{addtl_suggestion}".format(
+          port_flag=port_flag, addtl_suggestion=addtl_suggestion))))
+      print_to_stderr(str(e))
+      self.close_connection()
       raise
     except ImportError:
       print_to_stderr("Unable to import the python 'ssl' module. It is"
       " required for an SSL-secured connection.")
-      sys.exit(1)
+      raise FatalShellException()
     except socket.error, e:
       # if the socket was interrupted, reconnect the connection with the client
       if e.errno == errno.EINTR:
         self._reconnect_cancellation()
       else:
         print_to_stderr("Socket error %s: %s" % (e.errno, e))
-        self.imp_client.close_connection()
+        self.close_connection()
         self.prompt = self.DISCONNECTED_PROMPT
     except Exception, e:
       if self.ldap_password_cmd and \
@@ -847,7 +889,7 @@ class ImpalaShell(object, cmd.Cmd):
         print_to_stderr("Warning: TLSv1.2 is not supported for Python < 2.7.9")
       print_to_stderr("Error connecting: %s, %s" % (type(e).__name__, e))
       # A secure connection may still be open. So we explicitly close it.
-      self.imp_client.close_connection()
+      self.close_connection()
       # If a connection to another impalad failed while already connected
       # reset the prompt to disconnected.
       self.server_version = self.UNKNOWN_SERVER_VERSION
@@ -906,22 +948,22 @@ class ImpalaShell(object, cmd.Cmd):
       return db_table_name
 
   def do_alter(self, args):
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_create(self, args):
     # We want to print the webserver link only for CTAS queries.
     print_web_link = "select" in args
-    query = self._create_beeswax_query(args)
+    query = self._build_query_string(self.last_leading_comment, self.orig_cmd, args)
     return self._execute_stmt(query, print_web_link=print_web_link)
 
   def do_drop(self, args):
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_load(self, args):
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_profile(self, args):
     """Prints the runtime profile of the last DML statement or SELECT query executed."""
@@ -936,15 +978,15 @@ class ImpalaShell(object, cmd.Cmd):
 
   def do_select(self, args):
     """Executes a SELECT... query, fetching all rows"""
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query, print_web_link=True)
+    query_str = self._build_query_string(self.last_leading_comment, self.orig_cmd, args)
+    return self._execute_stmt(query_str, print_web_link=True)
 
   def do_compute(self, args):
     """Executes a COMPUTE STATS query.
     Impala shell cannot get child query handle so it cannot
     query live progress for COMPUTE STATS query. Disable live
     progress/summary callback for COMPUTE STATS query."""
-    query = self._create_beeswax_query(args)
+    query = self._build_query_string(self.last_leading_comment, self.orig_cmd, args)
     (prev_print_progress, prev_print_summary) = self.print_progress, self.print_summary
     (self.print_progress, self.print_summary) = False, False;
     try:
@@ -1015,22 +1057,22 @@ class ImpalaShell(object, cmd.Cmd):
                                              "#Rows", "Est. #Rows", "Peak Mem",
                                              "Est. Peak Mem", "Detail"])
 
-  def _execute_stmt(self, query, is_dml=False, print_web_link=False):
-    """ The logic of executing any query statement
-
-    The client executes the query and the query_handle is returned immediately,
-    even as the client waits for the query to finish executing.
+  def _execute_stmt(self, query_str, is_dml=False, print_web_link=False):
+    """Executes 'query_str' with options self.set_query_options on the Impala server.
+    The query is run to completion and close with any results, warnings, errors or
+    other output that we want to display to users printed to the output.
+    Results for queries are streamed from the server and displayed as they become
+    available.
 
-    If the query was not dml, the results are fetched from the client
-    as they are streamed in, through the use of a generator.
-
-    The execution time is printed and the query is closed if it hasn't been already
+    is_dml: True iff the caller detects that 'query_str' is a DML statement, false
+            otherwise.
+    print_web_link: if True, a link to the query on the Impala debug webserver is printed.
     """
 
-    self._print_if_verbose("Query: %s" % query.query)
+    self._print_if_verbose("Query: %s" % query_str)
     # TODO: Clean up this try block and refactor it (IMPALA-3814)
     try:
-      if self.webserver_address == ImpalaShell.UNKNOWN_WEBSERVER:
+      if not self.webserver_address:
         print_web_link = False
       if print_web_link:
         self._print_if_verbose("Query submitted at: %s (Coordinator: %s)" %
@@ -1038,13 +1080,15 @@ class ImpalaShell(object, cmd.Cmd):
             self.webserver_address))
 
       start_time = time.time()
-      self.last_query_handle = self.imp_client.execute_query(query)
-      self.query_handle_closed = False
+      self.last_query_handle = None
+      self.last_query_handle = self.imp_client.execute_query(
+          query_str, self.set_query_options)
       self.last_summary = time.time()
       if print_web_link:
         self._print_if_verbose(
             "Query progress can be monitored at: %s/query_plan?query_id=%s" %
-            (self.webserver_address, self.last_query_handle.id))
+            (self.webserver_address,
+             self.imp_client.get_query_id_str(self.last_query_handle)))
 
       wait_to_finish = self.imp_client.wait_to_finish(self.last_query_handle,
           self._periodic_wait_callback)
@@ -1057,10 +1101,9 @@ class ImpalaShell(object, cmd.Cmd):
         (num_rows, num_row_errors) = self.imp_client.close_dml(self.last_query_handle)
       else:
         # impalad does not support the fetching of metadata for certain types of queries.
-        if not self.imp_client.expect_result_metadata(query.query):
+        if not self.imp_client.expect_result_metadata(query_str, self.last_query_handle):
           # Close the query
           self.imp_client.close_query(self.last_query_handle)
-          self.query_handle_closed = True
           return CmdStatus.SUCCESS
 
         self._format_outputstream()
@@ -1097,8 +1140,7 @@ class ImpalaShell(object, cmd.Cmd):
           (verb, num_rows, error_report, time_elapsed))
 
       if not is_dml:
-        self.imp_client.close_query(self.last_query_handle, self.query_handle_closed)
-      self.query_handle_closed = True
+        self.imp_client.close_query(self.last_query_handle)
       try:
         profile = self.imp_client.get_runtime_profile(self.last_query_handle)
         self.print_runtime_profile(profile)
@@ -1112,7 +1154,8 @@ class ImpalaShell(object, cmd.Cmd):
       print_to_stderr(e)
     except QueryStateException, e:
       # an exception occurred while executing the query
-      self.imp_client.close_query(self.last_query_handle, self.query_handle_closed)
+      if self.last_query_handle is not None:
+        self.imp_client.close_query(self.last_query_handle)
       print_to_stderr(e)
     except DisconnectedException, e:
       # the client has lost the connection
@@ -1132,7 +1175,7 @@ class ImpalaShell(object, cmd.Cmd):
       # if the exception is unknown, there was possibly an issue with the connection
       # set the shell as disconnected
       print_to_stderr('Unknown Exception : %s' % (u,))
-      self.imp_client.connected = False
+      self.close_connection()
       self.prompt = ImpalaShell.DISCONNECTED_PROMPT
     return CmdStatus.ERROR
 
@@ -1151,15 +1194,15 @@ class ImpalaShell(object, cmd.Cmd):
 
   def do_values(self, args):
     """Executes a VALUES(...) query, fetching all rows"""
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_with(self, args):
     """Executes a query with a WITH clause, fetching all rows"""
-    query = self._create_beeswax_query(args)
+    query = self._build_query_string(self.last_leading_comment, self.orig_cmd, args)
     # Use shlex to deal with escape quotes in string literals.
     # Set posix=False to preserve the quotes.
-    tokens = shlex.split(strip_comments(query.query.lstrip()).encode('utf-8'),
+    tokens = shlex.split(strip_comments(query.lstrip()).encode('utf-8'),
                          posix=False)
     try:
       # Because the WITH clause may precede DML or SELECT queries,
@@ -1172,8 +1215,9 @@ class ImpalaShell(object, cmd.Cmd):
 
   def do_use(self, args):
     """Executes a USE... query"""
-    query = self._create_beeswax_query(args)
-    if self._execute_stmt(query) is CmdStatus.SUCCESS:
+    cmd_status = self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
+    if cmd_status is CmdStatus.SUCCESS:
       self.current_db = args.strip('`').strip()
       self.set_prompt(self.current_db)
     elif args.strip('`') == self.current_db:
@@ -1187,42 +1231,44 @@ class ImpalaShell(object, cmd.Cmd):
 
   def do_show(self, args):
     """Executes a SHOW... query, fetching all rows"""
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_describe(self, args):
+    return self.__do_describe(self.orig_cmd, args)
+
+  def do_desc(self, args):
+    return self.__do_describe("describe", args)
+
+  def __do_describe(self, cmd, args):
     """Executes a DESCRIBE... query, fetching all rows"""
     # original command should be overridden because the server cannot
     # recognize "desc" as a keyword. Thus, given command should be
     # replaced with "describe" here.
-    self.orig_cmd = "describe"
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
-
-  def do_desc(self, args):
-    return self.do_describe(args)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, cmd, args))
 
-  def __do_dml(self, args):
+  def __do_dml(self, orig_cmd, args):
     """Executes a DML query"""
-    query = self._create_beeswax_query(args)
+    query = self._build_query_string(self.last_leading_comment, orig_cmd, args)
     return self._execute_stmt(query, is_dml=True, print_web_link=True)
 
   def do_upsert(self, args):
-    return self.__do_dml(args)
+    return self.__do_dml(self.orig_cmd, args)
 
   def do_update(self, args):
-    return self.__do_dml(args)
+    return self.__do_dml(self.orig_cmd, args)
 
   def do_delete(self, args):
-    return self.__do_dml(args)
+    return self.__do_dml(self.orig_cmd, args)
 
   def do_insert(self, args):
-    return self.__do_dml(args)
+    return self.__do_dml(self.orig_cmd, args)
 
   def do_explain(self, args):
     """Explain the query execution plan"""
-    query = self._create_beeswax_query(args)
-    return self._execute_stmt(query)
+    return self._execute_stmt(
+        self._build_query_string(self.last_leading_comment, self.orig_cmd, args))
 
   def do_history(self, args):
     """Display command history"""
@@ -1406,9 +1452,10 @@ class ImpalaShell(object, cmd.Cmd):
     for history_item in history_items:
       self.readline.add_history(history_item)
 
-  def default(self, args):
-    query = self.imp_client.create_beeswax_query(args, self.set_query_options)
-    return self._execute_stmt(query, print_web_link=True)
+  def default(self, line):
+    """Called for any command that doesn't have a do_*() method. Sends the command
+    with any arguments and trailing comment to the server."""
+    return self._execute_stmt(line, print_web_link=True)
 
   def emptyline(self):
     """If an empty line is entered, do nothing"""
@@ -1522,7 +1569,7 @@ def parse_variables(keyvals):
         print_to_stderr('Error: Could not parse key-value "%s". ' % (keyval,) +
                         'It must follow the pattern "KEY=VALUE".')
         parser.print_help()
-        sys.exit(1)
+        raise FatalShellException()
       else:
         vars[match.groups()[0].upper()] = replace_variables(vars, match.groups()[1])
   return vars
@@ -1572,7 +1619,8 @@ def get_var_name(name):
   return None
 
 def execute_queries_non_interactive_mode(options, query_options):
-  """Run queries in non-interactive mode."""
+  """Run queries in non-interactive mode. Return True on success. Logs the
+  error and returns False otherwise."""
   if options.query_file:
     try:
       # "-" here signifies input from STDIN
@@ -1582,19 +1630,18 @@ def execute_queries_non_interactive_mode(options, query_options):
         query_file_handle = open(options.query_file, 'r')
     except Exception, e:
       print_to_stderr("Could not open file '%s': %s" % (options.query_file, e))
-      sys.exit(1)
+      return False
 
     query_text = query_file_handle.read()
   elif options.query:
     query_text = options.query
   else:
-    return
+    return True
 
   queries = parse_query_text(query_text)
-  shell = ImpalaShell(options, query_options)
-  if not (shell.execute_query_list(shell.cmdqueue) and
-          shell.execute_query_list(queries)):
-    sys.exit(1)
+  with ImpalaShell(options, query_options) as shell:
+    return (shell.execute_query_list(shell.cmdqueue) and
+            shell.execute_query_list(queries))
 
 def get_intro(options):
   """Get introduction message for start-up. The last character should not be a return."""
@@ -1608,7 +1655,8 @@ def get_intro(options):
               "not secured by TLS.\nALL PASSWORDS WILL BE SENT IN THE CLEAR TO IMPALA.")
   return intro
 
-if __name__ == "__main__":
+
+def impala_shell_main():
   """
   There are two types of options: shell options and query_options. Both can be set on the
   command line, which override default options. Specifically, if there exists a global
@@ -1620,6 +1668,7 @@ if __name__ == "__main__":
   changed in impala-shell with the 'set' command.
   """
   # pass defaults into option parser
+  global options, parser
   parser = get_option_parser(impala_shell_defaults)
   options, args = parser.parse_args()
 
@@ -1635,7 +1684,7 @@ if __name__ == "__main__":
         "Loading in options from global config file: %s \n" % global_config)
   elif global_config != impala_shell_defaults['global_config_default_path']:
     print_to_stderr('%s not found.\n' % global_config)
-    sys.exit(1)
+    raise FatalShellException()
   # Override the default user config by a custom config if necessary
   user_config = impala_shell_defaults.get("config_file")
   input_config = os.path.expanduser(options.config_file)
@@ -1648,7 +1697,7 @@ if __name__ == "__main__":
       user_config = input_config
     else:
       print_to_stderr('%s not found.\n' % input_config)
-      sys.exit(1)
+      raise FatalShellException()
   configs_to_load = [global_config, user_config]
 
   # load shell and query options from the list of config files
@@ -1665,7 +1714,7 @@ if __name__ == "__main__":
     impala_shell_defaults.update(loaded_shell_options)
   except Exception, e:
     print_to_stderr(e)
-    sys.exit(1)
+    raise FatalShellException()
 
   parser = get_option_parser(impala_shell_defaults)
   options, args = parser.parse_args()
@@ -1674,25 +1723,25 @@ if __name__ == "__main__":
   if len(args) > 0:
     print_to_stderr('Error, could not parse arguments "%s"' % (' ').join(args))
     parser.print_help()
-    sys.exit(1)
+    raise FatalShellException()
 
   if options.version:
     print VERSION_STRING
-    sys.exit(0)
+    return
 
   if options.use_kerberos and options.use_ldap:
     print_to_stderr("Please specify at most one authentication mechanism (-k or -l)")
-    sys.exit(1)
+    raise FatalShellException()
 
   if not options.ssl and not options.creds_ok_in_clear and options.use_ldap:
     print_to_stderr("LDAP credentials may not be sent over insecure " +
                     "connections. Enable SSL or set --auth_creds_ok_in_clear")
-    sys.exit(1)
+    raise FatalShellException()
 
   if not options.use_ldap and options.ldap_password_cmd:
     print_to_stderr("Option --ldap_password_cmd requires using LDAP authentication " +
                     "mechanism (-l)")
-    sys.exit(1)
+    raise FatalShellException()
 
   if options.use_kerberos:
     if options.verbose:
@@ -1703,10 +1752,10 @@ if __name__ == "__main__":
       if call(['klist', '-s']) != 0:
         print_to_stderr(("-k requires a valid kerberos ticket but no valid kerberos "
                          "ticket found."))
-        sys.exit(1)
+        raise FatalShellException()
     except OSError, e:
       print_to_stderr('klist not found on the system, install kerberos clients')
-      sys.exit(1)
+      raise FatalShellException()
   elif options.use_ldap:
     if options.verbose:
       print_to_stderr("Starting Impala Shell using LDAP-based authentication")
@@ -1723,11 +1772,11 @@ if __name__ == "__main__":
       if p.returncode != 0:
         print_to_stderr("Error retrieving LDAP password (command was '%s', error was: "
                         "'%s')" % (options.ldap_password_cmd, stderr.strip()))
-        sys.exit(1)
+        raise FatalShellException()
     except Exception, e:
       print_to_stderr("Error retrieving LDAP password (command was: '%s', exception "
                       "was: '%s')" % (options.ldap_password_cmd, e))
-      sys.exit(1)
+      raise FatalShellException()
 
   if options.ssl:
     if options.ca_cert is None:
@@ -1745,7 +1794,7 @@ if __name__ == "__main__":
       open(options.output_file, 'wb')
     except IOError, e:
       print_to_stderr('Error opening output file for writing: %s' % e)
-      sys.exit(1)
+      raise FatalShellException()
 
   options.variables = parse_variables(options.keyval)
 
@@ -1756,46 +1805,55 @@ if __name__ == "__main__":
   if options.query or options.query_file:
     if options.print_progress or options.print_summary:
       print_to_stderr("Error: Live reporting is available for interactive mode only.")
-      sys.exit(1)
+      raise FatalShellException()
 
-    execute_queries_non_interactive_mode(options, query_options)
-    sys.exit(0)
+    if execute_queries_non_interactive_mode(options, query_options):
+      return
+    else:
+      raise FatalShellException()
 
   intro = get_intro(options)
 
-  shell = ImpalaShell(options, query_options)
-  while shell.is_alive:
-    try:
+  with ImpalaShell(options, query_options) as shell:
+    while shell.is_alive:
       try:
-        shell.cmdloop(intro)
-      except KeyboardInterrupt:
-        print_to_stderr('^C')
-      # A last measure against any exceptions thrown by an rpc
-      # not caught in the shell
-      except socket.error, (code, e):
-        # if the socket was interrupted, reconnect the connection with the client
-        if code == errno.EINTR:
-          print shell.CANCELLATION_MESSAGE
-          shell._reconnect_cancellation()
-        else:
-          print_to_stderr("Socket error %s: %s" % (code, e))
+        try:
+          shell.cmdloop(intro)
+        except KeyboardInterrupt:
+          print_to_stderr('^C')
+        # A last measure against any exceptions thrown by an rpc
+        # not caught in the shell
+        except socket.error, (code, e):
+          # if the socket was interrupted, reconnect the connection with the client
+          if code == errno.EINTR:
+            print shell.CANCELLATION_MESSAGE
+            shell._reconnect_cancellation()
+          else:
+            print_to_stderr("Socket error %s: %s" % (code, e))
+            shell.imp_client.connected = False
+            shell.prompt = shell.DISCONNECTED_PROMPT
+        except DisconnectedException, e:
+          # the client has lost the connection
+          print_to_stderr(e)
           shell.imp_client.connected = False
           shell.prompt = shell.DISCONNECTED_PROMPT
-      except DisconnectedException, e:
-        # the client has lost the connection
-        print_to_stderr(e)
-        shell.imp_client.connected = False
-        shell.prompt = shell.DISCONNECTED_PROMPT
-      except QueryStateException, e:
-        # an exception occurred while executing the query
-        shell.imp_client.close_query(shell.last_query_handle,
-                                     shell.query_handle_closed)
-        print_to_stderr(e)
-      except RPCException, e:
-        # could not complete the rpc successfully
-        print_to_stderr(e)
-      except IOError, e:
-        # Interrupted system calls (e.g. because of cancellation) should be ignored.
-        if e.errno != errno.EINTR: raise
-    finally:
-      intro = ''
+        except QueryStateException, e:
+          # an exception occurred while executing the query
+          shell.imp_client.close_query(shell.last_query_handle)
+          print_to_stderr(e)
+        except RPCException, e:
+          # could not complete the rpc successfully
+          print_to_stderr(e)
+        except IOError, e:
+          # Interrupted system calls (e.g. because of cancellation) should be ignored.
+          if e.errno != errno.EINTR: raise
+      finally:
+        intro = ''
+
+
+if __name__ == "__main__":
+  try:
+    impala_shell_main()
+  except FatalShellException:
+    # Ensure that fatal errors cause a clean exit with error.
+    sys.exit(1)
diff --git a/shell/impala_shell_config_defaults.py b/shell/impala_shell_config_defaults.py
index 4ecf534..2b649b7 100644
--- a/shell/impala_shell_config_defaults.py
+++ b/shell/impala_shell_config_defaults.py
@@ -33,7 +33,7 @@ impala_shell_defaults = {
             'history_file': _histfile_from_env,
             'history_max': 1000,
             'ignore_query_failure': False,
-            'impalad': socket.getfqdn() + ':21000',
+            'impalad': socket.getfqdn(),
             'kerberos_host_fqdn': None,
             'kerberos_service_name': 'impala',
             'output_delimiter': '\\t',
diff --git a/shell/make_shell_tarball.sh b/shell/make_shell_tarball.sh
index dd6a648..2047bae 100755
--- a/shell/make_shell_tarball.sh
+++ b/shell/make_shell_tarball.sh
@@ -94,7 +94,12 @@ for MODULE in ${SHELL_HOME}/ext-py/*; do
   rm -rf dist 2>&1 > /dev/null
   rm -rf build 2>&1 > /dev/null
   echo "Creating an egg for ${MODULE}"
-  python setup.py -q bdist_egg clean
+  if [[ "$MODULE" == *"/bitarray"* ]]; then
+    # Need to use setuptools to build egg for bitarray module
+    python -c "import setuptools; execfile('setup.py')" -q bdist_egg clean
+  else
+    python setup.py -q bdist_egg clean
+  fi
   cp dist/*.egg ${TARBALL_ROOT}/ext-py
   popd 2>&1 > /dev/null
 done
diff --git a/shell/option_parser.py b/shell/option_parser.py
index b035967..cca97a4 100755
--- a/shell/option_parser.py
+++ b/shell/option_parser.py
@@ -257,6 +257,10 @@ def get_option_parser(defaults):
                     help="Timeout in milliseconds after which impala-shell will time out"
                     " if it fails to connect to Impala server. Set to 0 to disable any"
                     " timeout.")
+  parser.add_option("--protocol", dest="protocol", default="beeswax",
+                    help="Protocol to use for client/server connection - either 'hs2' or "
+                         "'beeswax'. Beeswax support is deprecated and will be removed "
+                         "in the future.")
 
   # add default values to the help text
   for option in parser.option_list:
diff --git a/shell/thrift_sasl.py b/shell/thrift_sasl.py
index 5cd02eb..c63c0d5 100644
--- a/shell/thrift_sasl.py
+++ b/shell/thrift_sasl.py
@@ -25,7 +25,6 @@ import sys
 from cStringIO import StringIO
 from thrift.transport import TTransport
 from thrift.transport.TTransport import *
-from thrift.protocol import TBinaryProtocol
 try:
   import saslwrapper as sasl
 except ImportError:
diff --git a/testdata/workloads/functional-query/queries/QueryTest/set.test b/testdata/workloads/functional-query/queries/QueryTest/set.test
index 12c7989..31ab8ef 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -7,7 +7,7 @@ set buffer_pool_limit=7;
 ---- QUERY
 set all;
 ---- RESULTS: VERIFY_IS_SUBSET
-'ABORT_ON_ERROR','0','REGULAR'
+'ABORT_ON_ERROR','1','REGULAR'
 'BATCH_SIZE','0','DEVELOPMENT'
 'BUFFER_POOL_LIMIT','','ADVANCED'
 'DEBUG_ACTION','','DEVELOPMENT'
@@ -33,7 +33,7 @@ STRING, STRING, STRING
 set explain_level=3;
 set all;
 ---- RESULTS: VERIFY_IS_SUBSET
-'ABORT_ON_ERROR','0','REGULAR'
+'ABORT_ON_ERROR','1','REGULAR'
 'BATCH_SIZE','0','DEVELOPMENT'
 'BUFFER_POOL_LIMIT','','ADVANCED'
 'DEBUG_ACTION','','DEVELOPMENT'
@@ -59,7 +59,7 @@ STRING, STRING, STRING
 set explain_level='0';
 set all;
 ---- RESULTS: VERIFY_IS_SUBSET
-'ABORT_ON_ERROR','0','REGULAR'
+'ABORT_ON_ERROR','1','REGULAR'
 'BATCH_SIZE','0','DEVELOPMENT'
 'BUFFER_POOL_LIMIT','','ADVANCED'
 'DEBUG_ACTION','','DEVELOPMENT'
diff --git a/tests/common/impala_service.py b/tests/common/impala_service.py
index 24eba8e..7697724 100644
--- a/tests/common/impala_service.py
+++ b/tests/common/impala_service.py
@@ -104,11 +104,13 @@ class BaseImpalaService(object):
       LOG.info("Sleeping %ds before next retry." % interval)
       sleep(interval)
     assert 0, 'Metric value %s did not reach value %s in %ss\nDumping impalad debug ' \
-              'pages:\nmemz: %s\nmetrics: %s\nqueries: %s\nthreadz: %s\nrpcz: %s' % \
+              'pages:\nmemz: %s\nmetrics: %s\nqueries: %s\nsessions: %s\nthreadz: %s\n '\
+              'rpcz: %s' % \
               (metric_name, expected_value, timeout,
                json.dumps(self.read_debug_webpage('memz?json')),
                json.dumps(self.read_debug_webpage('metrics?json')),
                json.dumps(self.read_debug_webpage('queries?json')),
+               json.dumps(self.read_debug_webpage('sessions?json')),
                json.dumps(self.read_debug_webpage('threadz?json')),
                json.dumps(self.read_debug_webpage('rpcz?json')))
 
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index 44e9124..5a89cf3 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -192,10 +192,7 @@ class ImpalaTestSuite(BaseTestSuite):
     # Cleanup the Impala and Hive Metastore client connections
     if cls.hive_transport:
       cls.hive_transport.close()
-    if cls.client:
-      cls.client.close()
-    if cls.hs2_client:
-      cls.hs2_client.close()
+    cls.close_impala_clients()
 
   @classmethod
   def create_impala_client(cls, host_port=None, protocol='beeswax',
@@ -209,6 +206,14 @@ class ImpalaTestSuite(BaseTestSuite):
     return client
 
   @classmethod
+  def close_impala_clients(cls):
+    """Close Impala clients created by setup_class()."""
+    if cls.client:
+      cls.client.close()
+    if cls.hs2_client:
+      cls.hs2_client.close()
+
+  @classmethod
   def __get_default_host_port(cls, protocol):
     if protocol == 'beeswax':
       return IMPALAD
diff --git a/tests/custom_cluster/test_shell_interactive.py b/tests/custom_cluster/test_shell_interactive.py
index 254d8e9..fa0c590 100644
--- a/tests/custom_cluster/test_shell_interactive.py
+++ b/tests/custom_cluster/test_shell_interactive.py
@@ -21,7 +21,7 @@ import os
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.test_vector import ImpalaTestVector
-from tests.common.test_dimensions import create_beeswax_dimension
+from tests.common.test_dimensions import create_beeswax_hs2_dimension
 from tests.shell.util import get_shell_cmd, get_impalad_port
 
 
@@ -39,7 +39,7 @@ class TestShellInteractive(CustomClusterTestSuite):
     self.client.wait_for_admission_control(sleep_query_handle)
 
     # Iterate over test vector within test function to avoid restarting cluster.
-    for vector in [ImpalaTestVector([value]) for value in create_beeswax_dimension()]:
+    for vector in [ImpalaTestVector([value]) for value in create_beeswax_hs2_dimension()]:
       cmd = get_shell_cmd(vector)
       proc = pexpect.spawn(cmd[0], cmd[1:])
       # Check with only live_summary set to true.
diff --git a/tests/custom_cluster/test_shell_interactive_reconnect.py b/tests/custom_cluster/test_shell_interactive_reconnect.py
index 66a9fcf..0370933 100644
--- a/tests/custom_cluster/test_shell_interactive_reconnect.py
+++ b/tests/custom_cluster/test_shell_interactive_reconnect.py
@@ -24,10 +24,11 @@ import os
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.impala_service import ImpaladService
 from tests.common.test_vector import ImpalaTestVector
-from tests.common.test_dimensions import create_beeswax_dimension
+from tests.common.test_dimensions import create_beeswax_hs2_dimension
 from tests.shell.util import ImpalaShell, get_shell_cmd, get_impalad_port
 # Follow tests/shell/test_shell_interactive.py naming.
 from shell.impala_shell import ImpalaShell as ImpalaShellClass
+from tests.verifiers.metric_verifier import MetricVerifier
 
 NUM_QUERIES = 'impala-server.num-queries'
 
@@ -40,7 +41,7 @@ class TestShellInteractiveReconnect(CustomClusterTestSuite):
   @pytest.mark.execute_serially
   def test_manual_reconnect(self):
     # Iterate over test vector within test function to avoid restarting cluster.
-    for vector in [ImpalaTestVector([value]) for value in create_beeswax_dimension()]:
+    for vector in [ImpalaTestVector([value]) for value in create_beeswax_hs2_dimension()]:
       p = ImpalaShell(vector)
       p.send_cmd("USE functional")
       # Connect without arguments works because the custom cluster will have the default
@@ -56,7 +57,7 @@ class TestShellInteractiveReconnect(CustomClusterTestSuite):
     impalad = ImpaladService(socket.getfqdn())
 
     # Iterate over test vector within test function to avoid restarting cluster.
-    for vector in [ImpalaTestVector([value]) for value in create_beeswax_dimension()]:
+    for vector in [ImpalaTestVector([value]) for value in create_beeswax_hs2_dimension()]:
       p = ImpalaShell(vector)
       # ImpalaShell startup may issue query to get server info - get num queries after
       # starting shell.
@@ -82,7 +83,7 @@ class TestShellInteractiveReconnect(CustomClusterTestSuite):
     impalad = ImpaladService(socket.getfqdn())
 
     # Iterate over test vector within test function to avoid restarting cluster.
-    for vector in [ImpalaTestVector([value]) for value in create_beeswax_dimension()]:
+    for vector in [ImpalaTestVector([value]) for value in create_beeswax_hs2_dimension()]:
       cmd = get_shell_cmd(vector)
       proc = pexpect.spawn(cmd[0], cmd[1:])
       proc.expect("{0}] default>".format(get_impalad_port(vector)))
@@ -106,3 +107,9 @@ class TestShellInteractiveReconnect(CustomClusterTestSuite):
       proc.sendline("show tables;")
       proc.expect("nation")
       proc.expect("{0}] tpch>".format(get_impalad_port(vector)))
+      proc.sendeof()
+      proc.wait()
+
+      # Ensure no sessions or queries are left dangling.
+      verifier = MetricVerifier(self.impalad_test_service)
+      verifier.verify_metrics_are_zero()
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index d50773f..3e6346f 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -114,7 +114,7 @@ class TestHS2(HS2TestSuite):
     # Verify that 'DEVELOPMENT' options are not returned.
     assert "DEBUG_ACTION" not in vals2
 
-    # Removed options should not be returned.
+    # Removed options are not returned.
     assert "MAX_IO_BUFFERS" not in vals2
 
   @needs_session()
@@ -133,9 +133,8 @@ class TestHS2(HS2TestSuite):
     assert levels["SYNC_DDL"] == "REGULAR"
     assert levels["MAX_ERRORS"] == "ADVANCED"
     assert levels["DEBUG_ACTION"] == "DEVELOPMENT"
-
-    # Removed options should not be returned.
-    assert "MAX_IO_BUFFERS" not in vals
+    # Removed options are returned by "SET ALL" for the benefit of impala-shell.
+    assert levels["MAX_IO_BUFFERS"] == "REMOVED"
 
   @SkipIfDockerizedCluster.internal_hostname
   def test_open_session_http_addr(self):
diff --git a/tests/query_test/test_observability.py b/tests/query_test/test_observability.py
index b26b7b2..005a661 100644
--- a/tests/query_test/test_observability.py
+++ b/tests/query_test/test_observability.py
@@ -190,18 +190,28 @@ class TestObservability(ImpalaTestSuite):
     assert "CLIENT_IDENTIFIER=" + \
         "query_test/test_observability.py::TestObservability::()::test_query_options" \
         in profile
+    # Get the TIMEZONE value.
+    server_timezone = None
+    for row in self.execute_query("set", query_opts).data:
+      name, val, _ = row.split("\t")
+      if name == "TIMEZONE":
+        server_timezone = val
+        break
+    assert server_timezone is not None
+
     # For this query, the planner sets NUM_NODES=1, NUM_SCANNER_THREADS=1,
     # RUNTIME_FILTER_MODE=0 and MT_DOP=0
     expected_str = ("Query Options (set by configuration and planner): "
         "MEM_LIMIT=8589934592,"
         "NUM_NODES=1,NUM_SCANNER_THREADS=1,"
-        "RUNTIME_FILTER_MODE=0,MT_DOP=0,{erasure_coding}"
+        "RUNTIME_FILTER_MODE=0,MT_DOP=0,TIMEZONE={timezone},{erasure_coding}"
         "CLIENT_IDENTIFIER="
         "query_test/test_observability.py::TestObservability::()::test_query_options"
         "\n")
     expected_str = expected_str.format(
-        erasure_coding="ALLOW_ERASURE_CODED_FILES=1," if IS_EC else "")
-    assert expected_str in profile
+        erasure_coding="ALLOW_ERASURE_CODED_FILES=1," if IS_EC else "",
+        timezone=server_timezone)
+    assert expected_str in profile, profile
 
   def test_exec_summary(self):
     """Test that the exec summary is populated correctly in every query state"""
diff --git a/tests/shell/test_shell_commandline.py b/tests/shell/test_shell_commandline.py
index 5491f31..2b490c0 100644
--- a/tests/shell/test_shell_commandline.py
+++ b/tests/shell/test_shell_commandline.py
@@ -30,7 +30,7 @@ from subprocess import call, Popen
 from tests.common.impala_service import ImpaladService
 from tests.common.impala_test_suite import ImpalaTestSuite, IMPALAD_HS2_HOST_PORT
 from tests.common.skip import SkipIf
-from tests.common.test_dimensions import create_beeswax_dimension
+from tests.common.test_dimensions import create_beeswax_hs2_dimension
 from time import sleep, time
 from util import (get_impalad_host_port, assert_var_substitution, run_impala_shell_cmd,
                   ImpalaShell, IMPALA_SHELL_EXECUTABLE)
@@ -117,7 +117,8 @@ class TestImpalaShell(ImpalaTestSuite):
 
   @classmethod
   def add_test_dimensions(cls):
-    cls.ImpalaTestMatrix.add_dimension(create_beeswax_dimension())
+    # Run with both beeswax and HS2 to ensure that behaviour is the same.
+    cls.ImpalaTestMatrix.add_dimension(create_beeswax_hs2_dimension())
 
   def test_no_args(self, vector):
     args = ['-q', DEFAULT_QUERY]
@@ -336,7 +337,7 @@ class TestImpalaShell(ImpalaTestSuite):
 
     args = ['-q', 'summary;']
     result_set = run_impala_shell_cmd(vector, args, expect_success=False)
-    assert "Could not retrieve summary for query" in result_set.stderr
+    assert "Could not retrieve summary: no previous query" in result_set.stderr
 
     args = ['-q', 'show tables; summary;']
     result_set = run_impala_shell_cmd(vector, args)
@@ -472,7 +473,13 @@ class TestImpalaShell(ImpalaTestSuite):
     output when pretty-printing falls back to delimited output."""
     args = ['-q', "select '{0}\\t'".format(RUSSIAN_CHARS.encode('utf-8'))]
     result = run_impala_shell_cmd(vector, args)
-    assert 'Reverting to tab delimited text' in result.stderr
+    protocol = vector.get_value('protocol')
+    if protocol == 'beeswax':
+      assert 'Reverting to tab delimited text' in result.stderr
+    else:
+      # HS2 does not need to fall back, but should behave appropriately.
+      assert protocol == 'hs2', protocol
+      assert 'Reverting to tab delimited text' not in result.stderr
     assert 'UnicodeDecodeError' not in result.stderr
     assert RUSSIAN_CHARS.encode('utf-8') in result.stdout
 
@@ -743,8 +750,12 @@ class TestImpalaShell(ImpalaTestSuite):
   def _validate_expected_socket_connected(self, vector, args, sock):
     # Building an one-off shell command instead of using Util::ImpalaShell since we need
     # to customize the impala daemon socket.
-    shell_cmd = [IMPALA_SHELL_EXECUTABLE]
-    expected_output = "PingImpalaService"
+    protocol = vector.get_value("protocol")
+    shell_cmd = [IMPALA_SHELL_EXECUTABLE, "--protocol={0}".format(protocol)]
+    if protocol == 'beeswax':
+      expected_output = "get_default_configuration"
+    else:
+      expected_output = "OpenSession"
     with open(os.devnull, 'w') as devnull:
       try:
         connection = None
@@ -908,8 +919,13 @@ class TestImpalaShell(ImpalaTestSuite):
     result = run_impala_shell_cmd(vector, ['-q', query, '-B'])
     # Note that Beeswax formats results differently (i.e. the "fix" for IMPALA-266 stopped
     # working for Beeswax at some point.
-    assert ("DOUBLE\t0.5\t8.071999999999999\t8\t8.071999999999999\t8\t8.072"
-            in result.stdout)
+    protocol = vector.get_value("protocol")
+    if protocol == 'hs2':
+      assert "DOUBLE\t0.5\t8.072\t8.0\t8.072\t8.0\t8.072" in result.stdout
+    else:
+      assert protocol == 'beeswax'
+      assert ("DOUBLE\t0.5\t8.071999999999999\t8\t8.071999999999999\t8\t8.072"
+              in result.stdout)
 
   def test_bool_display(self, vector):
     """Test that boolean values are displayed correctly."""
diff --git a/tests/shell/test_shell_interactive.py b/tests/shell/test_shell_interactive.py
index 5668a33..3edab72 100755
--- a/tests/shell/test_shell_interactive.py
+++ b/tests/shell/test_shell_interactive.py
@@ -37,9 +37,9 @@ from tempfile import NamedTemporaryFile
 from tests.common.impala_service import ImpaladService
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfLocal
-from tests.common.test_dimensions import create_beeswax_dimension
-from util import (assert_var_substitution, ImpalaShell, get_impalad_port,
-                  get_shell_cmd, get_open_sessions_metric)
+from tests.common.test_dimensions import create_beeswax_hs2_dimension
+from util import (assert_var_substitution, ImpalaShell, get_impalad_port, get_shell_cmd,
+                  get_open_sessions_metric)
 
 QUERY_FILE_PATH = os.path.join(os.environ['IMPALA_HOME'], 'tests', 'shell')
 
@@ -76,7 +76,8 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
 
   @classmethod
   def add_test_dimensions(cls):
-    cls.ImpalaTestMatrix.add_dimension(create_beeswax_dimension())
+    # Run with both beeswax and HS2 to ensure that behaviour is the same.
+    cls.ImpalaTestMatrix.add_dimension(create_beeswax_hs2_dimension())
 
   def _expect_with_cmd(self, proc, cmd, vector, expectations=(), db="default"):
     """Executes a command on the expect process instance and verifies a set of
@@ -87,6 +88,11 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
     for e in expectations:
       assert e in proc.before
 
+  def _wait_for_num_open_sessions(self, vector, impala_service, num, err):
+    """Helper method to wait for the number of open sessions to reach 'num'."""
+    metric_name = get_open_sessions_metric(vector)
+    assert impala_service.wait_for_metric_value(metric_name, num) == num, err
+
   def test_local_shell_options(self, vector):
     """Test that setting the local shell options works"""
     shell_cmd = get_shell_cmd(vector)
@@ -111,6 +117,8 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
     self._expect_with_cmd(proc, "set output_file=/tmp/clmn.txt", vector)
     self._expect_with_cmd(proc, "set", vector,
         ("DELIMITER: ,", "OUTPUT_FILE: /tmp/clmn.txt"))
+    proc.sendeof()
+    proc.wait()
 
   @pytest.mark.execute_serially
   def test_write_delimited(self, vector):
@@ -246,11 +254,6 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
     self.client.close()
     self.hs2_client.close()
 
-    def wait_for_num_open_sessions(impala_service, num, err):
-      """Helper method to wait for the number of open sessions to reach 'num'."""
-      metric_name = get_open_sessions_metric(vector)
-      assert impala_service.wait_for_metric_value(metric_name, num) == num, err
-
     hostname = socket.getfqdn()
     initial_impala_service = ImpaladService(hostname)
     target_impala_service = ImpaladService(hostname, webserver_port=25001,
@@ -261,25 +264,25 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
       target_port = 21001
     # This test is running serially, so there shouldn't be any open sessions, but wait
     # here in case a session from a previous test hasn't been fully closed yet.
-    wait_for_num_open_sessions(initial_impala_service, 0,
+    self._wait_for_num_open_sessions(vector, initial_impala_service, 0,
         "first impalad should not have any remaining open sessions.")
-    wait_for_num_open_sessions(target_impala_service, 0,
+    self._wait_for_num_open_sessions(vector, target_impala_service, 0,
         "second impalad should not have any remaining open sessions.")
     # Connect to the first impalad
     p = ImpalaShell(vector)
 
     # Make sure we're connected <hostname>:<port>
-    wait_for_num_open_sessions(initial_impala_service, 1,
+    self._wait_for_num_open_sessions(vector, initial_impala_service, 1,
         "Not connected to %s:%d" % (hostname, get_impalad_port(vector)))
     p.send_cmd("connect %s:%d" % (hostname, target_port))
 
     # The number of sessions on the target impalad should have been incremented.
-    wait_for_num_open_sessions(
+    self._wait_for_num_open_sessions(vector,
         target_impala_service, 1, "Not connected to %s:%d" % (hostname, target_port))
     assert "[%s:%d] default>" % (hostname, target_port) in p.get_result().stdout
 
     # The number of sessions on the initial impalad should have been decremented.
-    wait_for_num_open_sessions(initial_impala_service, 0,
+    self._wait_for_num_open_sessions(vector, initial_impala_service, 0,
         "Connection to %s:%d should have been closed" % (
           hostname, get_impalad_port(vector)))
 
@@ -314,6 +317,10 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
       impalad.wait_for_metric_value(NUM_QUERIES, start_num_queries + 5)
       assert impalad.wait_for_num_in_flight_queries(0), MSG % 'drop'
     finally:
+      # get_result() must be called to exit the shell.
+      p.get_result()
+      self._wait_for_num_open_sessions(vector, impalad, 0,
+          "shell should close sessions.")
       run_impala_shell_interactive(vector, "drop table if exists %s.%s;" % (
           TMP_DB, TMP_TBL))
       run_impala_shell_interactive(vector, "drop database if exists foo;")
@@ -342,6 +349,7 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
       child_proc.expect("Fetched 1 row\(s\) in [0-9]+\.?[0-9]*s")
     child_proc.expect(PROMPT_REGEX)
     child_proc.sendline('quit;')
+    child_proc.wait()
     p = ImpalaShell(vector)
     p.send_cmd('history')
     result = p.get_result()
@@ -400,6 +408,7 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
                           ("Command index to be rerun must be an integer."))
     self._expect_with_cmd(child_proc, "rerun1", vector, ("Syntax error"))
     child_proc.sendline('quit;')
+    child_proc.wait()
 
   def test_tip(self, vector):
     """Smoke test for the TIP command"""
@@ -710,6 +719,8 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
     self._expect_with_cmd(proc, "use foo", vector, (), 'default')
     self._expect_with_cmd(proc, "use functional", vector, (), 'functional')
     self._expect_with_cmd(proc, "use foo", vector, (), 'functional')
+    proc.sendeof()
+    proc.wait()
 
   def test_strip_leading_comment(self, vector):
     """Test stripping leading comments from SQL statements"""
diff --git a/tests/shell/util.py b/tests/shell/util.py
index b917ef7..030b753 100755
--- a/tests/shell/util.py
+++ b/tests/shell/util.py
@@ -27,7 +27,8 @@ from subprocess import Popen, PIPE
 
 from tests.common.environ import (IMPALA_LOCAL_BUILD_VERSION,
                                   IMPALA_TEST_CLUSTER_PROPERTIES)
-from tests.common.impala_test_suite import IMPALAD_BEESWAX_HOST_PORT
+from tests.common.impala_test_suite import (IMPALAD_BEESWAX_HOST_PORT,
+                                            IMPALAD_HS2_HOST_PORT)
 
 
 SHELL_HISTORY_FILE = os.path.expanduser("~/.impalahistory")
@@ -132,8 +133,11 @@ def run_impala_shell_cmd_no_expect(vector, shell_args, env=None, stdin_input=Non
 def get_impalad_host_port(vector):
   """Get host and port to connect to based on test vector provided."""
   protocol = vector.get_value("protocol")
-  assert protocol == 'beeswax', protocol
-  return IMPALAD_BEESWAX_HOST_PORT
+  if protocol == 'hs2':
+    return IMPALAD_HS2_HOST_PORT
+  else:
+    assert protocol == 'beeswax', protocol
+    return IMPALAD_BEESWAX_HOST_PORT
 
 
 def get_impalad_port(vector):
@@ -144,17 +148,18 @@ def get_impalad_port(vector):
 def get_shell_cmd(vector):
   """Get the basic shell command to start the shell, given the provided test vector.
   Returns the command as a list of string arguments."""
-  # Use impala-shell build instead of bin/impala-shell.sh so that we test with the
-  # system python, not the toolchain python and in a configuration close to what
-  # we will distribute.
-  return [IMPALA_SHELL_EXECUTABLE, "-i{0}".format(get_impalad_host_port(vector))]
+  return [IMPALA_SHELL_EXECUTABLE, "--protocol={0}".format(vector.get_value("protocol")),
+          "-i{0}".format(get_impalad_host_port(vector))]
 
 
 def get_open_sessions_metric(vector):
   """Get the name of the vector that tracks open sessions for the protocol in vector."""
   protocol = vector.get_value("protocol")
-  assert protocol == 'beeswax', protocol
-  return 'impala-server.num-open-beeswax-sessions'
+  if protocol == 'hs2':
+    return 'impala-server.num-open-hiveserver2-sessions'
+  else:
+    assert protocol == 'beeswax', protocol
+    return 'impala-server.num-open-beeswax-sessions'
 
 class ImpalaShellResult(object):
   def __init__(self):
diff --git a/tests/verifiers/metric_verifier.py b/tests/verifiers/metric_verifier.py
index 747c7ee..d44b629 100644
--- a/tests/verifiers/metric_verifier.py
+++ b/tests/verifiers/metric_verifier.py
@@ -30,8 +30,9 @@ METRIC_LIST = [
                # Buffer pool pages belong to specific queries. Therefore there should be
                # no clean pages if there are no queries running.
                "buffer-pool.clean-pages",
-               "buffer-pool.clean-page-bytes"
-               ]
+               "buffer-pool.clean-page-bytes",
+               "impala-server.num-open-beeswax-sessions",
+               "impala-server.num-open-hiveserver2-sessions"]
 
 class MetricVerifier(object):
   """Reuseable class that can verify common metrics"""
diff --git a/tests/verifiers/test_verify_metrics.py b/tests/verifiers/test_verify_metrics.py
index 3b96f9c..d1564c5 100644
--- a/tests/verifiers/test_verify_metrics.py
+++ b/tests/verifiers/test_verify_metrics.py
@@ -28,6 +28,12 @@ class TestValidateMetrics(ImpalaTestSuite):
   TODO: Add a test for local assignments.
   """
 
+  @classmethod
+  def setup_class(cls):
+    super(TestValidateMetrics, cls).setup_class()
+    # Close clients to make sure no sessions are held open by this test.
+    cls.close_impala_clients()
+
   def test_metrics_are_zero(self):
     """Test that all the metric in METRIC_LIST are 0"""
     verifier = MetricVerifier(self.impalad_test_service)