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 2018/04/11 05:34:45 UTC

[1/6] impala git commit: IMPALA-6215: Removes race when using LibCache.

Repository: impala
Updated Branches:
  refs/heads/master 830e3346f -> 818cd8fa2


IMPALA-6215: Removes race when using LibCache.

Re-do the previously reverted change for IMPALA-6215. This patch
addresses the flakes listed in IMPALA-6092, which have become more
urgent recently.

LibCache's api to provide access to locally cached files has a race.
Currently, the client of the cache accesses the locally cached path
as a string, but nothing guarantees that the associated file is not
removed before the client is done using it. This race is suspected
as the root cause for the flakiness seen in IMPALA-6092. These tests
fail once in a while with classloader errors unable to load java udf
classes. In these tests, the lib cache makes no guarantee that the path
to the jar will remain valid from the time the path is acquired through
the time needed to fetch the jar and resolve the needed classes.

LibCache offers liveness guarantees for shared objects via reference
counting. The fix in this patch extends this API to also cover paths
to locally cached files.

This fix *only* addresses the path race. General cleanup of the api will
be done separately.

Testing:
   - added a test to test_udfs.py that does many concurrent udf uses and
     removals. By increasing the concurrent operations to 100, the issue
     in IMPALA-6092 is locally reproducible on every run. With this fix,
     the problem is no longer reproducible with this test.

Change-Id: I72ac0dfb13cf37d79e25c5b8a258b65f2dad097f
Reviewed-on: http://gerrit.cloudera.org:8080/9968
Reviewed-by: Vuk Ercegovac <ve...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 27c028f0578ac5edb42ba55c2c54ab8b7c195422
Parents: 830e334
Author: Vuk Ercegovac <ve...@cloudera.com>
Authored: Tue Nov 21 08:41:03 2017 -0800
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Apr 11 02:14:28 2018 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen.cc               |  5 +-
 be/src/exec/external-data-source-executor.cc |  6 +-
 be/src/exprs/hive-udf-call.cc                | 57 +++++++--------
 be/src/exprs/hive-udf-call.h                 |  3 -
 be/src/runtime/lib-cache.cc                  | 18 +++--
 be/src/runtime/lib-cache.h                   | 39 +++++++++--
 be/src/service/fe-support.cc                 | 10 +--
 tests/query_test/test_udfs.py                | 84 +++++++++++++++++++++--
 8 files changed, 167 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 5ac19ad..c8fd8eb 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -339,9 +339,10 @@ Status LlvmCodeGen::LinkModuleFromLocalFs(const string& file) {
 
 Status LlvmCodeGen::LinkModuleFromHdfs(const string& hdfs_location, const time_t mtime) {
   if (linked_modules_.find(hdfs_location) != linked_modules_.end()) return Status::OK();
+  LibCacheEntryHandle handle;
   string local_path;
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      hdfs_location, LibCache::TYPE_IR, mtime, &local_path));
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
+      hdfs_location, LibCache::TYPE_IR, mtime, &handle, &local_path));
   RETURN_IF_ERROR(LinkModuleFromLocalFs(local_path));
   linked_modules_.insert(hdfs_location);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/exec/external-data-source-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/external-data-source-executor.cc b/be/src/exec/external-data-source-executor.cc
index e8217b7..5c06d3a 100644
--- a/be/src/exec/external-data-source-executor.cc
+++ b/be/src/exec/external-data-source-executor.cc
@@ -136,12 +136,12 @@ ExternalDataSourceExecutor::~ExternalDataSourceExecutor() {
 Status ExternalDataSourceExecutor::Init(const string& jar_path,
     const string& class_name, const string& api_version, const string& init_string) {
   DCHECK(!is_initialized_);
+  LibCacheEntryHandle handle;
   string local_jar_path;
   // TODO(IMPALA-6727): pass the mtime from the coordinator. for now, skip the mtime
   // check (-1).
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      jar_path, LibCache::TYPE_JAR, -1, &local_jar_path));
-
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
+      jar_path, LibCache::TYPE_JAR, -1, &handle, &local_jar_path));
   JNIEnv* jni_env = getJNIEnv();
 
   // Add a scoped cleanup jni reference object. This cleans up local refs made below.

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/exprs/hive-udf-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc
index be3965a..b7715b7 100644
--- a/be/src/exprs/hive-udf-call.cc
+++ b/be/src/exprs/hive-udf-call.cc
@@ -174,10 +174,6 @@ Status HiveUdfCall::Init(const RowDescriptor& row_desc, RuntimeState* state) {
   // Initialize children first.
   RETURN_IF_ERROR(ScalarExpr::Init(row_desc, state));
 
-  // Copy the Hive Jar from hdfs to local file system.
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      fn_.hdfs_location, LibCache::TYPE_JAR, fn_.last_modified_time, &local_location_));
-
   // Initialize input_byte_offsets_ and input_buffer_size_
   for (int i = 0; i < GetNumChildren(); ++i) {
     input_byte_offsets_.push_back(input_buffer_size_);
@@ -202,30 +198,35 @@ Status HiveUdfCall::OpenEvaluator(FunctionContext::FunctionStateScope scope,
   JNIEnv* env = getJNIEnv();
   if (env == NULL) return Status("Failed to get/create JVM");
 
-  THiveUdfExecutorCtorParams ctor_params;
-  ctor_params.fn = fn_;
-  ctor_params.local_location = local_location_;
-  ctor_params.input_byte_offsets = input_byte_offsets_;
-
-  jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
-  jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
-  jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
-
-  ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
-  ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
-  ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
-  ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
-
-  jbyteArray ctor_params_bytes;
-
-  // Add a scoped cleanup jni reference object. This cleans up local refs made
-  // below.
-  JniLocalFrame jni_frame;
-  RETURN_IF_ERROR(jni_frame.push(env));
-
-  RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
-  // Create the java executor object
-  jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
+  {
+    LibCacheEntryHandle handle;
+    string local_location;
+    RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(fn_.hdfs_location,
+        LibCache::TYPE_JAR, fn_.last_modified_time, &handle, &local_location));
+    THiveUdfExecutorCtorParams ctor_params;
+    ctor_params.fn = fn_;
+    ctor_params.local_location = local_location;
+    ctor_params.input_byte_offsets = input_byte_offsets_;
+
+    jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
+    jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
+    jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
+
+    ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
+    ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
+    ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
+    ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
+
+    jbyteArray ctor_params_bytes;
+
+    // Add a scoped cleanup jni reference object. This cleans up local refs made below.
+    JniLocalFrame jni_frame;
+    RETURN_IF_ERROR(jni_frame.push(env));
+
+    RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
+    // Create the java executor object
+    jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
+  }
   RETURN_ERROR_IF_EXC(env);
   RETURN_IF_ERROR(JniUtil::LocalToGlobalRef(env, jni_ctx->executor, &jni_ctx->executor));
 

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/exprs/hive-udf-call.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.h b/be/src/exprs/hive-udf-call.h
index 7ce5eb0..8ca0372 100644
--- a/be/src/exprs/hive-udf-call.h
+++ b/be/src/exprs/hive-udf-call.h
@@ -116,9 +116,6 @@ class HiveUdfCall : public ScalarExpr {
   /// error.
   AnyVal* Evaluate(ScalarExprEvaluator* eval, const TupleRow* row) const;
 
-  /// The path on the local FS to the UDF's jar
-  std::string local_location_;
-
   /// input_byte_offsets_[i] is the byte offset child ith's input argument should
   /// be written to.
   std::vector<int> input_byte_offsets_;

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/runtime/lib-cache.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.cc b/be/src/runtime/lib-cache.cc
index 83bb4dc..cb1bc74 100644
--- a/be/src/runtime/lib-cache.cc
+++ b/be/src/runtime/lib-cache.cc
@@ -131,6 +131,10 @@ LibCacheEntry::~LibCacheEntry() {
   unlink(local_path.c_str());
 }
 
+LibCacheEntryHandle::~LibCacheEntryHandle() {
+  if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry_);
+}
+
 Status LibCache::GetSoFunctionPtr(const string& hdfs_lib_file, const string& symbol,
     time_t exp_mtime, void** fn_ptr, LibCacheEntry** ent, bool quiet) {
   if (hdfs_lib_file.empty()) {
@@ -181,14 +185,16 @@ void LibCache::DecrementUseCount(LibCacheEntry* entry) {
   if (can_delete) delete entry;
 }
 
-Status LibCache::GetLocalLibPath(
-    const string& hdfs_lib_file, LibType type, time_t exp_mtime, string* local_path) {
-  unique_lock<mutex> lock;
+Status LibCache::GetLocalPath(const std::string& hdfs_lib_file, LibType type,
+    time_t exp_mtime, LibCacheEntryHandle* handle, string* path) {
+  DCHECK(handle != nullptr && handle->entry() == nullptr);
   LibCacheEntry* entry = nullptr;
+  unique_lock<mutex> lock;
   RETURN_IF_ERROR(GetCacheEntry(hdfs_lib_file, type, exp_mtime, &lock, &entry));
   DCHECK(entry != nullptr);
-  DCHECK_EQ(entry->type, type);
-  *local_path = entry->local_path;
+  ++entry->use_count;
+  handle->SetEntry(entry);
+  *path = entry->local_path;
   return Status::OK();
 }
 
@@ -422,7 +428,7 @@ Status LibCache::RefreshCacheEntry(const string& hdfs_lib_file, LibType type,
 
     // Let the caller propagate any error that occurred when loading the entry.
     RETURN_IF_ERROR((*entry)->copy_file_status);
-    DCHECK_EQ((*entry)->type, type);
+    DCHECK_EQ((*entry)->type, type) << (*entry)->local_path;
     DCHECK(!(*entry)->local_path.empty());
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/runtime/lib-cache.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h
index 820a1a8..b8a2981 100644
--- a/be/src/runtime/lib-cache.h
+++ b/be/src/runtime/lib-cache.h
@@ -49,11 +49,16 @@ class RuntimeState;
 /// using the library. When the caller requests a ptr into the library, they
 /// are given the entry handle and must decrement the ref count when they
 /// are done.
+/// Note: Explicitly managing this reference count at the client is error-prone. See the
+/// api for accessing a path, GetLocalPath(), that uses the handle's scope to manage the
+/// reference count.
 //
 /// TODO:
 /// - refresh libraries
-/// - better cached module management.
+/// - better cached module management
+/// - improve the api to be less error-prone (IMPALA-6439)
 struct LibCacheEntry;
+class LibCacheEntryHandle;
 
 class LibCache {
  public:
@@ -71,15 +76,19 @@ class LibCache {
   /// Initializes the libcache. Must be called before any other APIs.
   static Status Init();
 
-  /// Gets the local file system path for the library at 'hdfs_lib_file'. If
+  /// Gets the local 'path' used to cache the file stored at the global 'hdfs_lib_file'. If
   /// this file is not already on the local fs, or if the cached entry's last modified
   /// is older than expected mtime, 'exp_mtime', it copies it and caches the result.
   /// An 'exp_mtime' of -1 makes the mtime check a no-op.
+  ///
+  /// 'handle' must remain in scope while 'path' is used. The reference count to the
+  /// underlying cache entry is decremented when 'handle' goes out-of-scope.
+  ///
   /// Returns an error if 'hdfs_lib_file' cannot be copied to the local fs or if
   /// exp_mtime differs from the mtime on the file system.
   /// If error is due to refresh, then the entry will be removed from the cache.
-  Status GetLocalLibPath(const std::string& hdfs_lib_file, LibType type, time_t exp_mtime,
-      std::string* local_path);
+  Status GetLocalPath(const std::string& hdfs_lib_file, LibType type, time_t exp_mtime,
+      LibCacheEntryHandle* handle, string* path);
 
   /// Returns status.ok() if the symbol exists in 'hdfs_lib_file', non-ok otherwise.
   /// If status.ok() is true, 'mtime' is set to the cache entry's last modified time.
@@ -107,6 +116,7 @@ class LibCache {
   /// An 'exp_mtime' of -1 makes the mtime check a no-op.
   /// An error is returned if exp_mtime differs from the mtime on the file system.
   /// If error is due to refresh, then the entry will be removed from the cache.
+  /// TODO: api is error-prone. upgrade to LibCacheEntryHandle (see IMPALA-6439).
   Status GetSoFunctionPtr(const std::string& hdfs_lib_file, const std::string& symbol,
       time_t exp_mtime, void** fn_ptr, LibCacheEntry** entry, bool quiet = false);
 
@@ -204,6 +214,27 @@ class LibCache {
       const std::string& hdfs_lib_file, const LibMap::iterator& entry_iterator);
 };
 
+/// Handle for a LibCacheEntry that decrements its reference count when the handle is
+/// destroyed or re-used for another entry.
+class LibCacheEntryHandle {
+ public:
+  LibCacheEntryHandle() {}
+  ~LibCacheEntryHandle();
+
+ private:
+  friend class LibCache;
+
+  LibCacheEntry* entry() const { return entry_; }
+  void SetEntry(LibCacheEntry* entry) {
+    if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry);
+    entry_ = entry;
+  }
+
+  LibCacheEntry* entry_ = nullptr;
+
+  DISALLOW_COPY_AND_ASSIGN(LibCacheEntryHandle);
+};
+
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index a8906a0..187d14e 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -300,9 +300,10 @@ static void ResolveSymbolLookup(const TSymbolLookupParams params,
       // Refresh the library if necessary.
       LibCache::instance()->SetNeedsRefresh(params.location);
     }
+    LibCacheEntryHandle handle;
     string dummy_local_path;
-    Status status = LibCache::instance()->GetLocalLibPath(
-        params.location, type, -1, &dummy_local_path);
+    Status status = LibCache::instance()->GetLocalPath(
+        params.location, type, -1, &handle, &dummy_local_path);
     if (!status.ok()) {
       result->__set_result_code(TSymbolLookupResultCode::BINARY_NOT_FOUND);
       result->__set_error_msg(status.GetDetail());
@@ -397,10 +398,11 @@ Java_org_apache_impala_service_FeSupport_NativeCacheJar(
       JniUtil::internal_exc_class(), nullptr);
 
   TCacheJarResult result;
+  LibCacheEntryHandle handle;
   string local_path;
   // TODO(IMPALA-6727): used for external data sources; add proper mtime.
-  Status status = LibCache::instance()->GetLocalLibPath(
-      params.hdfs_location, LibCache::TYPE_JAR, -1, &local_path);
+  Status status = LibCache::instance()->GetLocalPath(
+      params.hdfs_location, LibCache::TYPE_JAR, -1, &handle, &local_path);
   status.ToThrift(&result.status);
   if (status.ok()) result.__set_local_path(local_path);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/27c028f0/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 644d80f..dc5491b 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -394,8 +394,6 @@ class TestUdfExecution(TestUdfBase):
       self.run_test_case('QueryTest/udf-non-deterministic', vector,
           use_db=unique_database)
 
-  # Runs serially as a temporary workaround for IMPALA_6092.
-  @pytest.mark.execute_serially
   def test_java_udfs(self, vector, unique_database):
     self.run_test_case('QueryTest/load-java-udfs', vector, use_db=unique_database)
     self.run_test_case('QueryTest/java-udf', vector, use_db=unique_database)
@@ -507,9 +505,6 @@ class TestUdfTargeted(TestUdfBase):
             unique_database, tgt_udf_path))
     query = "select `{0}`.fn_invalid_symbol('test')".format(unique_database)
 
-    # Dropping the function can interact with other tests whose Java classes are in
-    # the same jar. Use a copy of the jar to avoid unintended interactions.
-    # See IMPALA-6215 and IMPALA-6092 for examples.
     check_call(["hadoop", "fs", "-put", "-f", src_udf_path, tgt_udf_path])
     self.client.execute(drop_fn_stmt)
     self.client.execute(create_fn_stmt)
@@ -518,6 +513,85 @@ class TestUdfTargeted(TestUdfBase):
       assert "Unable to find class" in str(ex)
     self.client.execute(drop_fn_stmt)
 
+  def test_concurrent_jar_drop_use(self, vector, unique_database):
+    """IMPALA-6215: race between dropping/using java udf's defined in the same jar.
+       This test runs concurrent drop/use threads that result in class not found
+       exceptions when the race is present.
+    """
+    udf_src_path = os.path.join(
+      os.environ['IMPALA_HOME'], "testdata/udfs/impala-hive-udfs.jar")
+    udf_tgt_path = get_fs_path(
+      '/test-warehouse/{0}.db/impala-hive-udfs.jar'.format(unique_database))
+
+    create_fn_to_drop = """create function {0}.foo_{1}() returns string
+                           LOCATION '{2}' SYMBOL='org.apache.impala.TestUpdateUdf'"""
+    create_fn_to_use = """create function {0}.use_it(string) returns string
+                          LOCATION '{1}' SYMBOL='org.apache.impala.TestUdf'"""
+    drop_fn = "drop function if exists {0}.foo_{1}()"
+    use_fn = """select * from (select max(int_col) from functional.alltypesagg
+                where {0}.use_it(string_col) = 'blah' union all
+                (select max(int_col) from functional.alltypesagg
+                 where {0}.use_it(String_col) > '1' union all
+                (select max(int_col) from functional.alltypesagg
+                 where {0}.use_it(string_col) > '1'))) v"""
+    num_drops = 100
+    num_uses = 100
+
+    # use a unique jar for this test to avoid interactions with other tests
+    # that use the same jar
+    check_call(["hadoop", "fs", "-put", "-f", udf_src_path, udf_tgt_path])
+
+    # create all the functions.
+    setup_client = self.create_impala_client()
+    try:
+      s = create_fn_to_use.format(unique_database, udf_tgt_path)
+      setup_client.execute(s)
+    except Exception as e:
+      print e
+      assert False
+    for i in range(0, num_drops):
+      try:
+        setup_client.execute(create_fn_to_drop.format(unique_database, i, udf_tgt_path))
+      except Exception as e:
+        print e
+        assert False
+
+    errors = []
+    def use_fn_method():
+      time.sleep(5 + random.random())
+      client = self.create_impala_client()
+      try:
+        client.execute(use_fn.format(unique_database))
+      except Exception as e: errors.append(e)
+
+    def drop_fn_method(i):
+      time.sleep(1 + random.random())
+      client = self.create_impala_client()
+      try:
+        client.execute(drop_fn.format(unique_database, i))
+      except Exception as e: errors.append(e)
+
+    # create threads to use functions.
+    runner_threads = []
+    for i in range(0, num_uses):
+      runner_threads.append(threading.Thread(target=use_fn_method))
+
+    # create threads to drop functions.
+    drop_threads = []
+    for i in range(0, num_drops):
+      runner_threads.append(threading.Thread(target=drop_fn_method, args=(i, )))
+
+    # launch all runner threads.
+    for t in runner_threads: t.start()
+
+    # join all threads.
+    for t in runner_threads: t.join();
+
+    # Check for any errors.
+    for e in errors: print e
+    assert len(errors) == 0
+
+
   @SkipIfLocal.multiple_impalad
   def test_hive_udfs_missing_jar(self, vector, unique_database):
     """ IMPALA-2365: Impalad shouldn't crash if the udf jar isn't present


[3/6] impala git commit: IMPALA-5903: Inconsistent specification of result set and result set metadata

Posted by ta...@apache.org.
IMPALA-5903: Inconsistent specification of result set and result set metadata

Before this commit it was quite random which DDL oprations
returned a result set and which didn't.

With this commit, every DDL operations return a summary of
its execution. They declare their result set schema in
Frontend.java, and provide the summary in CalatogOpExecutor.java.

Updated the tests according to the new behavior.

Change-Id: Ic542fb8e49e850052416ac663ee329ee3974e3b9
Reviewed-on: http://gerrit.cloudera.org:8080/9090
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 2ee914d5b365c8230645fdd0604a67eff1edbeb2
Parents: 27c028f
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
Authored: Thu Apr 5 14:54:27 2018 +0200
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Apr 11 02:21:48 2018 +0000

----------------------------------------------------------------------
 be/src/service/client-request-state.cc          |   7 +-
 .../impala/service/CatalogOpExecutor.java       | 165 ++++++++++++++-----
 .../org/apache/impala/service/Frontend.java     |  25 +--
 shell/impala_client.py                          |   2 +-
 .../queries/QueryTest/alter-table.test          |  29 ++++
 .../queries/QueryTest/chars-tmp-tables.test     |   5 +
 .../QueryTest/compute-stats-incremental.test    |   1 +
 .../queries/QueryTest/compute-stats.test        |  13 +-
 .../queries/QueryTest/create-database.test      |   7 +
 .../QueryTest/create-table-like-file.test       |  14 ++
 .../QueryTest/create-table-like-table.test      |  11 ++
 .../queries/QueryTest/create-table.test         |  15 ++
 .../queries/QueryTest/delimited-latin-text.test |   3 +-
 .../queries/QueryTest/delimited-text.test       |   3 +-
 .../queries/QueryTest/describe-path.test        |   1 +
 .../queries/QueryTest/functions-ddl.test        |  16 ++
 .../queries/QueryTest/grant_revoke.test         |  25 +++
 .../queries/QueryTest/grant_revoke_kudu.test    |   4 +
 .../hbase-compute-stats-incremental.test        |   1 +
 .../queries/QueryTest/insert_bad_expr.test      |   2 +
 .../queries/QueryTest/insert_permutation.test   |   2 +
 .../queries/QueryTest/kudu_alter.test           |  18 ++
 .../queries/QueryTest/kudu_create.test          |  10 +-
 .../queries/QueryTest/kudu_delete.test          |   2 +
 .../queries/QueryTest/kudu_insert.test          |   5 +
 .../queries/QueryTest/kudu_partition_ddl.test   |  10 ++
 .../queries/QueryTest/kudu_stats.test           |   1 +
 .../queries/QueryTest/kudu_update.test          |   1 +
 .../queries/QueryTest/kudu_upsert.test          |   1 +
 .../QueryTest/libs_with_same_filenames.test     |   1 +
 .../queries/QueryTest/load.test                 |   2 +
 .../queries/QueryTest/local-filesystem.test     |   1 +
 .../partition-ddl-predicates-all-fs.test        |   4 +-
 .../QueryTest/test-unmatched-schema.test        |   5 +
 .../queries/QueryTest/truncate-table.test       |   2 +
 .../queries/QueryTest/udf-errors.test           |   7 +
 .../queries/QueryTest/views-ddl.test            |  17 ++
 37 files changed, 366 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/be/src/service/client-request-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 2aedcab..a8b9bfa 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -516,6 +516,8 @@ Status ClientRequestState::ExecDdlRequest() {
 
     if (child_queries.size() > 0) {
       RETURN_IF_ERROR(child_query_executor_->ExecAsync(move(child_queries)));
+    } else {
+      SetResultSet({"No partitions selected for incremental stats update."});
     }
     return Status::OK();
   }
@@ -646,6 +648,9 @@ void ClientRequestState::Wait() {
     discard_result(UpdateQueryStatus(status));
   }
   if (status.ok()) {
+    if (stmt_type() == TStmtType::DDL) {
+      DCHECK(catalog_op_type() != TCatalogOpType::DDL || request_result_set_ != nullptr);
+    }
     UpdateNonErrorOperationState(TOperationState::FINISHED_STATE);
   }
   // UpdateQueryStatus() or UpdateNonErrorOperationState() have updated operation_state_.
@@ -681,7 +686,7 @@ Status ClientRequestState::WaitInternal() {
 
   if (!returns_result_set()) {
     // Queries that do not return a result are finished at this point. This includes
-    // DML operations and a subset of the DDL operations.
+    // DML operations.
     eos_ = true;
   } else if (catalog_op_type() == TCatalogOpType::DDL &&
       ddl_type() == TDdlType::CREATE_TABLE_AS_SELECT) {

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 184a839..87513aa 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -133,6 +133,7 @@ import org.apache.impala.thrift.TPartitionDef;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TPartitionStats;
 import org.apache.impala.thrift.TPrivilege;
+import org.apache.impala.thrift.TRangePartitionOperationType;
 import org.apache.impala.thrift.TResetMetadataRequest;
 import org.apache.impala.thrift.TResetMetadataResponse;
 import org.apache.impala.thrift.TResultRow;
@@ -345,6 +346,21 @@ public class CatalogOpExecutor {
   }
 
   /**
+   * Create result set from string 'summary', and attach it to 'response'.
+   */
+  private void addSummary(TDdlExecResponse response, String summary) {
+    TColumnValue resultColVal = new TColumnValue();
+    resultColVal.setString_val(summary);
+    TResultSet resultSet = new TResultSet();
+    resultSet.setSchema(new TResultSetMetadata(Lists.newArrayList(new TColumn(
+        "summary", Type.STRING.toThrift()))));
+    TResultRow resultRow = new TResultRow();
+    resultRow.setColVals(Lists.newArrayList(resultColVal));
+    resultSet.setRows(Lists.newArrayList(resultRow));
+    response.setResult_set(resultSet);
+  }
+
+  /**
    * Execute the ALTER TABLE command according to the TAlterTableParams and refresh the
    * table metadata, except for RENAME, ADD PARTITION and DROP PARTITION. This call is
    * thread-safe, i.e. concurrent operations on the same table are serialized.
@@ -356,9 +372,6 @@ public class CatalogOpExecutor {
     // When true, loads the table schema and the column stats from the Hive Metastore.
     boolean reloadTableSchema = false;
 
-    // When true, sets the result to be reported to the client.
-    boolean setResultSet = false;
-    TColumnValue resultColVal = new TColumnValue();
     Reference<Long> numUpdatedPartitions = new Reference<>(0L);
 
     TableName tableName = TableName.fromThrift(params.getTable_name());
@@ -402,6 +415,11 @@ public class CatalogOpExecutor {
           alterTableAddReplaceCols(tbl, addReplaceColParams.getColumns(),
               addReplaceColParams.isReplace_existing_cols());
           reloadTableSchema = true;
+          if (addReplaceColParams.isReplace_existing_cols()) {
+            addSummary(response, "Table columns have been replaced.");
+          } else {
+            addSummary(response, "New column(s) have been added to the table.");
+          }
           break;
         case ADD_PARTITION:
           // Create and add HdfsPartition objects to the corresponding HdfsTable and load
@@ -413,17 +431,20 @@ public class CatalogOpExecutor {
             addTableToCatalogUpdate(refreshedTable, response.result);
           }
           reloadMetadata = false;
+          addSummary(response, "New partition has been added to the table.");
           break;
         case DROP_COLUMN:
           TAlterTableDropColParams dropColParams = params.getDrop_col_params();
           alterTableDropCol(tbl, dropColParams.getCol_name());
           reloadTableSchema = true;
+          addSummary(response, "Column has been dropped.");
           break;
         case ALTER_COLUMN:
           TAlterTableAlterColParams alterColParams = params.getAlter_col_params();
           alterTableAlterCol(tbl, alterColParams.getCol_name(),
               alterColParams.getNew_col_def());
           reloadTableSchema = true;
+          addSummary(response, "Column has been altered.");
           break;
         case DROP_PARTITION:
           TAlterTableDropPartitionParams dropPartParams =
@@ -440,9 +461,8 @@ public class CatalogOpExecutor {
             refreshedTable.setCatalogVersion(newCatalogVersion);
             addTableToCatalogUpdate(refreshedTable, response.result);
           }
-          resultColVal.setString_val(
+          addSummary(response,
               "Dropped " + numUpdatedPartitions.getRef() + " partition(s).");
-          setResultSet = true;
           reloadMetadata = false;
           break;
         case RENAME_TABLE:
@@ -458,12 +478,11 @@ public class CatalogOpExecutor {
               fileFormatParams.getFile_format(), numUpdatedPartitions);
 
           if (fileFormatParams.isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case SET_ROW_FORMAT:
           TAlterTableSetRowFormatParams rowFormatParams =
@@ -472,29 +491,33 @@ public class CatalogOpExecutor {
               rowFormatParams.getPartition_set(), rowFormatParams.getRow_format(),
               numUpdatedPartitions);
           if (rowFormatParams.isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case SET_LOCATION:
           TAlterTableSetLocationParams setLocationParams =
               params.getSet_location_params();
-          reloadFileMetadata = alterTableSetLocation(tbl,
-              setLocationParams.getPartition_spec(), setLocationParams.getLocation());
+          List<TPartitionKeyValue> partitionSpec = setLocationParams.getPartition_spec();
+          reloadFileMetadata = alterTableSetLocation(tbl, partitionSpec,
+              setLocationParams.getLocation());
+          if (partitionSpec == null) {
+            addSummary(response, "New location has been set.");
+          } else {
+            addSummary(response, "New location has been set for the specified partition.");
+          }
           break;
         case SET_TBL_PROPERTIES:
           alterTableSetTblProperties(tbl, params.getSet_tbl_properties_params(),
               numUpdatedPartitions);
           if (params.getSet_tbl_properties_params().isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case UPDATE_STATS:
           Preconditions.checkState(params.isSetUpdate_stats_params());
@@ -502,9 +525,8 @@ public class CatalogOpExecutor {
           alterTableUpdateStats(tbl, params.getUpdate_stats_params(),
               numUpdatedPartitions, numUpdatedColumns);
           reloadTableSchema = true;
-          resultColVal.setString_val("Updated " + numUpdatedPartitions.getRef() +
+          addSummary(response, "Updated " + numUpdatedPartitions.getRef() +
               " partition(s) and " + numUpdatedColumns.getRef() + " column(s).");
-          setResultSet = true;
           break;
         case SET_CACHED:
           Preconditions.checkState(params.isSetSet_cached_params());
@@ -513,17 +535,17 @@ public class CatalogOpExecutor {
           if (params.getSet_cached_params().getPartition_set() == null) {
             reloadFileMetadata =
                 alterTableSetCached(tbl, params.getSet_cached_params());
-            resultColVal.setString_val(op + "table.");
+            addSummary(response, op + "table.");
           } else {
             alterPartitionSetCached(tbl, params.getSet_cached_params(),
                 numUpdatedPartitions);
-            resultColVal.setString_val(
+            addSummary(response,
                 op + numUpdatedPartitions.getRef() + " partition(s).");
           }
-          setResultSet = true;
           break;
         case RECOVER_PARTITIONS:
           alterTableRecoverPartitions(tbl);
+          addSummary(response, "Partitions have been recovered.");
           break;
         default:
           throw new UnsupportedOperationException(
@@ -535,16 +557,6 @@ public class CatalogOpExecutor {
             reloadTableSchema, null);
         addTableToCatalogUpdate(tbl, response.result);
       }
-
-      if (setResultSet) {
-        TResultSet resultSet = new TResultSet();
-        resultSet.setSchema(new TResultSetMetadata(Lists.newArrayList(
-            new TColumn("summary", Type.STRING.toThrift()))));
-        TResultRow resultRow = new TResultRow();
-        resultRow.setColVals(Lists.newArrayList(resultColVal));
-        resultSet.setRows(Lists.newArrayList(resultRow));
-        response.setResult_set(resultSet);
-      }
     } finally {
       context.stop();
       Preconditions.checkState(!catalog_.getLock().isWriteLockedByCurrentThread());
@@ -575,21 +587,27 @@ public class CatalogOpExecutor {
             params.getAdd_replace_cols_params();
         KuduCatalogOpExecutor.addColumn((KuduTable) tbl,
             addReplaceColParams.getColumns());
+        addSummary(response, "Column has been added/replaced.");
         break;
       case DROP_COLUMN:
         TAlterTableDropColParams dropColParams = params.getDrop_col_params();
         KuduCatalogOpExecutor.dropColumn((KuduTable) tbl,
             dropColParams.getCol_name());
+        addSummary(response, "Column has been dropped.");
         break;
       case ALTER_COLUMN:
         TAlterTableAlterColParams alterColParams = params.getAlter_col_params();
         KuduCatalogOpExecutor.alterColumn((KuduTable) tbl, alterColParams.getCol_name(),
             alterColParams.getNew_col_def());
+        addSummary(response, "Column has been altered.");
         break;
       case ADD_DROP_RANGE_PARTITION:
         TAlterTableAddDropRangePartitionParams partParams =
             params.getAdd_drop_range_partition_params();
         KuduCatalogOpExecutor.addDropRangePartition((KuduTable) tbl, partParams);
+        addSummary(response, "Range partition has been " +
+            (partParams.type == TRangePartitionOperationType.ADD ?
+            "added." : "dropped."));
         break;
       default:
         throw new UnsupportedOperationException(
@@ -692,6 +710,7 @@ public class CatalogOpExecutor {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         tbl.load(true, msClient.getHiveClient(), msTbl);
       }
+      addSummary(resp, "View has been altered.");
       tbl.setCatalogVersion(newCatalogVersion);
       addTableToCatalogUpdate(tbl, resp.result);
     } finally {
@@ -946,6 +965,7 @@ public class CatalogOpExecutor {
       Preconditions.checkNotNull(existingDb);
       resp.getResult().addToUpdated_catalog_objects(existingDb.toTCatalogObject());
       resp.getResult().setVersion(existingDb.getCatalogVersion());
+      addSummary(resp, "Database already exists.");
       return;
     }
     org.apache.hadoop.hive.metastore.api.Database db =
@@ -966,11 +986,13 @@ public class CatalogOpExecutor {
         try {
           msClient.getHiveClient().createDatabase(db);
           newDb = catalog_.addDb(dbName, db);
+          addSummary(resp, "Database has been created.");
         } catch (AlreadyExistsException e) {
           if (!params.if_not_exists) {
             throw new ImpalaRuntimeException(
                 String.format(HMS_RPC_ERROR_FORMAT_STR, "createDatabase"), e);
           }
+          addSummary(resp, "Database already exists.");
           if (LOG.isTraceEnabled()) {
             LOG.trace(String.format("Ignoring '%s' when creating database %s because " +
                 "IF NOT EXISTS was specified.", e, dbName));
@@ -1022,6 +1044,7 @@ public class CatalogOpExecutor {
             throw new CatalogException("Function " + fn.functionName() +
                 " already exists.");
           }
+          addSummary(resp, "Function already exists.");
           return;
         }
       }
@@ -1059,6 +1082,9 @@ public class CatalogOpExecutor {
       if (!addedFunctions.isEmpty()) {
         resp.result.setUpdated_catalog_objects(addedFunctions);
         resp.result.setVersion(catalog_.getCatalogVersion());
+        addSummary(resp, "Function has been created.");
+      } else {
+        addSummary(resp, "Function already exists.");
       }
     }
   }
@@ -1073,6 +1099,7 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException("Data source " + dataSource.getName() +
             " already exists.");
       }
+      addSummary(resp, "Data source already exists.");
       resp.result.addToUpdated_catalog_objects(existingDataSource.toTCatalogObject());
       resp.result.setVersion(existingDataSource.getCatalogVersion());
       return;
@@ -1080,6 +1107,7 @@ public class CatalogOpExecutor {
     catalog_.addDataSource(dataSource);
     resp.result.addToUpdated_catalog_objects(dataSource.toTCatalogObject());
     resp.result.setVersion(dataSource.getCatalogVersion());
+    addSummary(resp, "Data source has been created.");
   }
 
   private void dropDataSource(TDropDataSourceParams params, TDdlExecResponse resp)
@@ -1091,12 +1119,14 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException("Data source " + params.getData_source() +
             " does not exists.");
       }
+      addSummary(resp, "Data source does not exist.");
       // No data source was removed.
       resp.result.setVersion(catalog_.getCatalogVersion());
       return;
     }
     resp.result.addToRemoved_catalog_objects(dataSource.toTCatalogObject());
     resp.result.setVersion(dataSource.getCatalogVersion());
+    addSummary(resp, "Data source has been dropped.");
   }
 
   /**
@@ -1126,7 +1156,10 @@ public class CatalogOpExecutor {
         HdfsTable hdfsTbl = (HdfsTable) table;
         List<HdfsPartition> partitions =
             hdfsTbl.getPartitionsFromPartitionSet(params.getPartition_set());
-        if (partitions.isEmpty()) return;
+        if (partitions.isEmpty()) {
+          addSummary(resp, "No partitions found for table.");
+          return;
+        }
 
         for(HdfsPartition partition : partitions) {
           if (partition.getPartitionStats() != null) {
@@ -1141,6 +1174,7 @@ public class CatalogOpExecutor {
       }
       loadTableMetadata(table, newCatalogVersion, false, true, null);
       addTableToCatalogUpdate(table, resp.result);
+      addSummary(resp, "Stats have been dropped.");
     } finally {
       Preconditions.checkState(!catalog_.getLock().isWriteLockedByCurrentThread());
       table.getLock().unlock();
@@ -1258,11 +1292,13 @@ public class CatalogOpExecutor {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().dropDatabase(
             params.getDb(), true, params.if_exists, params.cascade);
+        addSummary(resp, "Database has been dropped.");
       } catch (TException e) {
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "dropDatabase"), e);
       }
       Db removedDb = catalog_.removeDb(params.getDb());
+
       if (removedDb == null) {
         // Nothing was removed from the catalogd's cache.
         resp.result.setVersion(catalog_.getCatalogVersion());
@@ -1277,6 +1313,7 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(removedObject);
     resp.result.setVersion(removedObject.getCatalog_version());
     resp.result.addToRemoved_catalog_objects(removedObject);
+    addSummary(resp, "Database has been dropped.");
   }
 
   /**
@@ -1333,14 +1370,20 @@ public class CatalogOpExecutor {
     synchronized (metastoreDdlLock_) {
       Db db = catalog_.getDb(params.getTable_name().db_name);
       if (db == null) {
-        if (params.if_exists) return;
-        throw new CatalogException("Database does not exist: " +
-            params.getTable_name().db_name);
+        String dbNotExist = "Database does not exist: " + params.getTable_name().db_name;
+        if (params.if_exists) {
+          addSummary(resp, dbNotExist);
+          return;
+        }
+        throw new CatalogException(dbNotExist);
       }
       Table existingTbl = db.getTable(params.getTable_name().table_name);
       if (existingTbl == null) {
-        if (params.if_exists) return;
-        throw new CatalogException("Table/View does not exist: " + tableName);
+        if (params.if_exists) {
+          addSummary(resp, (params.is_table ? "Table " : "View ") + "does not exist.");
+          return;
+        }
+        throw new CatalogException("Table/View does not exist.");
       }
 
       // Retrieve the HMS table to determine if this is a Kudu table.
@@ -1366,9 +1409,13 @@ public class CatalogOpExecutor {
       // fixed.
       if (params.isSetIs_table() && ((params.is_table && existingTbl instanceof View)
           || (!params.is_table && !(existingTbl instanceof View)))) {
-        if (params.if_exists) return;
         String errorMsg = "DROP " + (params.is_table ? "TABLE " : "VIEW ") +
             "not allowed on a " + (params.is_table ? "view: " : "table: ") + tableName;
+        if (params.if_exists) {
+          addSummary(resp, "Drop " + (params.is_table ? "table " : "view ") +
+              "is not allowed on a " + (params.is_table ? "view." : "table."));
+          return;
+        }
         throw new CatalogException(errorMsg);
       }
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
@@ -1382,6 +1429,7 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "dropTable"), e);
       }
+      addSummary(resp, (params.is_table ? "Table " : "View ") + "has been dropped.");
 
       Table table = catalog_.removeTable(params.getTable_name().db_name,
           params.getTable_name().table_name);
@@ -1442,7 +1490,10 @@ public class CatalogOpExecutor {
     try {
       table = getExistingTable(tblName.getDb_name(), tblName.getTable_name());
     } catch (TableNotFoundException e) {
-      if (params.if_exists) return;
+      if (params.if_exists) {
+        addSummary(resp, "Table does not exist.");
+        return;
+      }
       throw e;
     }
     Preconditions.checkNotNull(table);
@@ -1472,6 +1523,7 @@ public class CatalogOpExecutor {
         throw new CatalogException(String.format("Failed to truncate table: %s.\n" +
             "Table may be in a partially truncated state.", fqName), e);
       }
+      addSummary(resp, "Table has been truncated.");
 
       loadTableMetadata(table, newCatalogVersion, true, true, null);
       addTableToCatalogUpdate(table, resp.result);
@@ -1491,6 +1543,7 @@ public class CatalogOpExecutor {
             throw new CatalogException("Database: " + fName.getDb()
                 + " does not exist.");
         }
+        addSummary(resp, "Database does not exist.");
         return;
       }
       List<TCatalogObject> removedFunctions = Lists.newArrayList();
@@ -1524,7 +1577,10 @@ public class CatalogOpExecutor {
       }
 
       if (!removedFunctions.isEmpty()) {
+        addSummary(resp, "Function has been dropped.");
         resp.result.setRemoved_catalog_objects(removedFunctions);
+      } else {
+        addSummary(resp, "Function does not exist.");
       }
       resp.result.setVersion(catalog_.getCatalogVersion());
     }
@@ -1546,6 +1602,7 @@ public class CatalogOpExecutor {
 
     Table existingTbl = catalog_.getTableNoThrow(tableName.getDb(), tableName.getTbl());
     if (params.if_not_exists && existingTbl != null) {
+      addSummary(response, "Table already exists.");
       LOG.trace(String.format("Skipping table creation because %s already exists and " +
           "IF NOT EXISTS was specified.", tableName));
       existingTbl.getLock().lock();
@@ -1667,10 +1724,14 @@ public class CatalogOpExecutor {
             " dropped. The log contains more information.", newTable.getTableName(),
             kuduTableName), e);
       }
-      if (e instanceof AlreadyExistsException && params.if_not_exists) return false;
+      if (e instanceof AlreadyExistsException && params.if_not_exists) {
+        addSummary(response, "Table already exists.");
+        return false;
+      }
       throw new ImpalaRuntimeException(
           String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
     }
+    addSummary(response, "Table has been created.");
     return true;
   }
 
@@ -1687,6 +1748,7 @@ public class CatalogOpExecutor {
     synchronized (metastoreDdlLock_) {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().createTable(newTable);
+        addSummary(response, "Table has been created.");
         // If this table should be cached, and the table location was not specified by
         // the user, an extra step is needed to read the table to find the location.
         if (cacheOp != null && cacheOp.isSet_cached() &&
@@ -1695,7 +1757,10 @@ public class CatalogOpExecutor {
               newTable.getDbName(), newTable.getTableName());
         }
       } catch (Exception e) {
-        if (e instanceof AlreadyExistsException && if_not_exists) return false;
+        if (e instanceof AlreadyExistsException && if_not_exists) {
+          addSummary(response, "Table already exists");
+          return false;
+        }
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
       }
@@ -1739,7 +1804,11 @@ public class CatalogOpExecutor {
         new org.apache.hadoop.hive.metastore.api.Table();
     setViewAttributes(params, view);
     LOG.trace(String.format("Creating view %s", tableName));
-    createTable(view, params.if_not_exists, null, response);
+    if (!createTable(view, params.if_not_exists, null, response)) {
+      addSummary(response, "View already exists.");
+    } else {
+      addSummary(response, "View has been created.");
+    }
   }
 
   /**
@@ -1762,6 +1831,7 @@ public class CatalogOpExecutor {
 
     Table existingTbl = catalog_.getTableNoThrow(tblName.getDb(), tblName.getTbl());
     if (params.if_not_exists && existingTbl != null) {
+      addSummary(response, "Table already exists.");
       LOG.trace(String.format("Skipping table creation because %s already exists and " +
           "IF NOT EXISTS was specified.", tblName));
       existingTbl.getLock().lock();
@@ -2226,6 +2296,7 @@ public class CatalogOpExecutor {
     response.result.addToRemoved_catalog_objects(result.first.toMinimalTCatalogObject());
     response.result.addToUpdated_catalog_objects(result.second.toTCatalogObject());
     response.result.setVersion(result.second.getCatalogVersion());
+    addSummary(response, "Renaming was successful.");
   }
 
   /**
@@ -2838,6 +2909,7 @@ public class CatalogOpExecutor {
       if (role == null) {
         // Nothing was removed from the catalogd's cache.
         resp.result.setVersion(catalog_.getCatalogVersion());
+        addSummary(resp, "No such role.");
         return;
       }
     } else {
@@ -2852,8 +2924,10 @@ public class CatalogOpExecutor {
     catalogObject.setCatalog_version(role.getCatalogVersion());
     if (createDropRoleParams.isIs_drop()) {
       resp.result.addToRemoved_catalog_objects(catalogObject);
+      addSummary(resp, "Role has been dropped.");
     } else {
       resp.result.addToUpdated_catalog_objects(catalogObject);
+      addSummary(resp, "Role has been created.");
     }
     resp.result.setVersion(role.getCatalogVersion());
   }
@@ -2884,6 +2958,11 @@ public class CatalogOpExecutor {
     catalogObject.setRole(role.toThrift());
     catalogObject.setCatalog_version(role.getCatalogVersion());
     resp.result.addToUpdated_catalog_objects(catalogObject);
+    if (grantRevokeRoleParams.isIs_grant()) {
+      addSummary(resp, "Role has been granted.");
+    } else {
+      addSummary(resp, "Role has been revoked.");
+    }
     resp.result.setVersion(role.getCatalogVersion());
   }
 
@@ -2902,9 +2981,11 @@ public class CatalogOpExecutor {
     if (grantRevokePrivParams.isIs_grant()) {
       rolePrivileges = catalog_.getSentryProxy().grantRolePrivileges(requestingUser,
           roleName, privileges);
+      addSummary(resp, "Privilege(s) have been granted.");
     } else {
       rolePrivileges = catalog_.getSentryProxy().revokeRolePrivileges(requestingUser,
           roleName, privileges, grantRevokePrivParams.isHas_grant_opt());
+      addSummary(resp, "Privilege(s) have been revoked.");
     }
     Preconditions.checkNotNull(rolePrivileges);
     List<TCatalogObject> updatedPrivs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 539fe31..392c249 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -334,21 +334,18 @@ public class Frontend {
       req.setDdl_type(TDdlType.ALTER_TABLE);
       req.setAlter_table_params(analysis.getAlterTableStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isAlterViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.ALTER_VIEW);
       req.setAlter_view_params(analysis.getAlterViewStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateTableStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_TABLE);
       req.setCreate_table_params(analysis.getCreateTableStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateTableAsSelectStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -356,29 +353,24 @@ public class Frontend {
       req.setCreate_table_params(
           analysis.getCreateTableAsSelectStmt().getCreateStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Arrays.asList(
-          new TColumn("summary", Type.STRING.toThrift())));
     } else if (analysis.isCreateTableLikeStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_TABLE_LIKE);
       req.setCreate_table_like_params(analysis.getCreateTableLikeStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_VIEW);
       req.setCreate_view_params(analysis.getCreateViewStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateDbStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_DATABASE);
       req.setCreate_db_params(analysis.getCreateDbStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateUdfStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       CreateUdfStmt stmt = (CreateUdfStmt) analysis.getStmt();
@@ -386,7 +378,6 @@ public class Frontend {
       req.setDdl_type(TDdlType.CREATE_FUNCTION);
       req.setCreate_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateUdaStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -394,7 +385,6 @@ public class Frontend {
       CreateUdaStmt stmt = (CreateUdaStmt)analysis.getStmt();
       req.setCreate_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateDataSrcStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -402,21 +392,18 @@ public class Frontend {
       CreateDataSrcStmt stmt = (CreateDataSrcStmt)analysis.getStmt();
       req.setCreate_data_source_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isComputeStatsStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.COMPUTE_STATS);
       req.setCompute_stats_params(analysis.getComputeStatsStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropDbStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.DROP_DATABASE);
       req.setDrop_db_params(analysis.getDropDbStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropTableOrViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -424,7 +411,6 @@ public class Frontend {
       req.setDdl_type(stmt.isDropTable() ? TDdlType.DROP_TABLE : TDdlType.DROP_VIEW);
       req.setDrop_table_or_view_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isTruncateStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -432,7 +418,6 @@ public class Frontend {
       req.setDdl_type(TDdlType.TRUNCATE_TABLE);
       req.setTruncate_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropFunctionStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -440,7 +425,6 @@ public class Frontend {
       DropFunctionStmt stmt = (DropFunctionStmt)analysis.getStmt();
       req.setDrop_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropDataSrcStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -448,7 +432,6 @@ public class Frontend {
       DropDataSrcStmt stmt = (DropDataSrcStmt)analysis.getStmt();
       req.setDrop_data_source_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropStatsStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -456,7 +439,6 @@ public class Frontend {
       DropStatsStmt stmt = (DropStatsStmt) analysis.getStmt();
       req.setDrop_stats_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isResetMetadataStmt()) {
       ddl.op_type = TCatalogOpType.RESET_METADATA;
       ResetMetadataStmt resetMetadataStmt = (ResetMetadataStmt) analysis.getStmt();
@@ -499,7 +481,6 @@ public class Frontend {
       req.setCreate_drop_role_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isGrantRevokeRoleStmt()) {
       GrantRevokeRoleStmt grantRoleStmt = (GrantRevokeRoleStmt) analysis.getStmt();
       TGrantRevokeRoleParams params = grantRoleStmt.toThrift();
@@ -508,7 +489,6 @@ public class Frontend {
       req.setGrant_revoke_role_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isGrantRevokePrivStmt()) {
       GrantRevokePrivStmt grantRevokePrivStmt = (GrantRevokePrivStmt) analysis.getStmt();
       TGrantRevokePrivParams params = grantRevokePrivStmt.toThrift();
@@ -518,10 +498,13 @@ public class Frontend {
       req.setGrant_revoke_priv_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else {
       throw new IllegalStateException("Unexpected CatalogOp statement type.");
     }
+    // All DDL commands return a string summarizing the outcome of the DDL.
+    if (ddl.op_type == TCatalogOpType.DDL) {
+      metadata.setColumns(Arrays.asList(new TColumn("summary", Type.STRING.toThrift())));
+    }
     result.setResult_set_metadata(metadata);
     ddl.setSync_ddl(result.getQuery_options().isSync_ddl());
     result.setCatalog_op_request(ddl);

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/shell/impala_client.py
----------------------------------------------------------------------
diff --git a/shell/impala_client.py b/shell/impala_client.py
index d4bfbee..5fa50b0 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -486,7 +486,7 @@ class ImpalaClient(object):
 
   def expect_result_metadata(self, query_str):
     """ Given a query string, return True if impalad expects result metadata"""
-    excluded_query_types = ['use', 'drop']
+    excluded_query_types = ['use']
     if True in set(map(query_str.startswith, excluded_query_types)):
       return False
     return True

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
index 555a599..013b8a9 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
@@ -4,6 +4,7 @@
 create external table t1 (i int)
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t1_tmp1'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe t1
@@ -16,6 +17,9 @@ string,string,string
 # Add some columns
 alter table t1 add columns (t tinyint, s string comment 'Str Col')
 ---- RESULTS
+'New column(s) have been added to the table.'
+---- TYPES
+string
 ====
 ---- QUERY
 describe t1
@@ -29,6 +33,7 @@ string,string,string
 ---- QUERY
 alter table t1 rename to t2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 show tables
@@ -42,6 +47,7 @@ string
 # Move the table to a different database
 alter table t2 rename to $DATABASE2.t1_inotherdb
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # No longer appears in this database
@@ -63,6 +69,7 @@ string
 # Move the table back to this database
 alter table $DATABASE2.t1_inotherdb rename to t2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # make sure the new table shows the same columns as the old table
@@ -77,6 +84,7 @@ string,string,string
 ---- QUERY
 alter table t2 drop column t
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 # The dropped column no longer shows up
@@ -91,6 +99,7 @@ string,string,string
 # Replace the columns with new values
 alter table t2 replace columns (c1 bigint comment 'id col', c2 string, c3 int)
 ---- RESULTS
+'Table columns have been replaced.'
 ====
 ---- QUERY
 describe t2
@@ -118,14 +127,17 @@ bigint,string,int
 ---- QUERY
 alter table t2 change column c2 int_col int comment 'changed to int col'
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 alter table t2 change column c1 id_col bigint
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 alter table t2 change column c3 c3 int comment 'added a comment'
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 describe t2
@@ -151,6 +163,7 @@ x array<int>,
 y map<string,float> comment 'Map Col',
 z struct<f1:boolean,f2:bigint>)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 describe t2
@@ -171,6 +184,7 @@ a int comment 'Int Col',
 b struct<f1:array<int>,f2:map<string,struct<f1:bigint>>>,
 c double)
 ---- RESULTS
+'Table columns have been replaced.'
 ====
 ---- QUERY
 describe t2
@@ -217,6 +231,7 @@ string,string,string
 ---- QUERY
 create external table jointbl_test like functional.jointbl
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Expect new table to be empty
@@ -282,6 +297,7 @@ alter table t_part add if not exists partition (j=1, s='2012');
 alter table t_part add if not exists partition (j=1, s='2012/withslash');
 alter table t_part add partition (j=1, s=substring('foo2013bar', 4, 8));
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Add another partition that points to the same location as another partition.
@@ -289,6 +305,7 @@ alter table t_part add partition (j=1, s=substring('foo2013bar', 4, 8));
 alter table t_part add partition (j=100, s='same_location')
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t_part_tmp/j=1/s=2012'
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Add another partition that points to an existing data location that does not
@@ -296,6 +313,7 @@ location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t_part_tmp/j=1/s=2012'
 alter table t_part add partition (j=101, s='different_part_dir')
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/part_data/'
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 insert overwrite table t_part partition(j=1, s='2012') select 2 from functional.alltypes limit 2
@@ -337,10 +355,12 @@ int,int,string
 ---- QUERY
 alter table t_part add partition (j=NULL, s='2013')
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 alter table t_part add partition (j=NULL, s=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Drop the partition that points to a duplication location. The data will no longer
@@ -527,6 +547,7 @@ int,int,string
 # rename a partitioned table
 alter table t_part rename to t_part2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # only the new table shows up
@@ -558,6 +579,7 @@ int
 alter table alltypes_test partition(month=4, year=2009)
 set location '$FILESYSTEM_PREFIX/test-warehouse/alltypes_seq_snap/year=2009/month=4'
 ---- RESULTS
+'New location has been set for the specified partition.'
 ====
 ---- QUERY
 alter table alltypes_test partition(month=4, year=2009)
@@ -590,6 +612,7 @@ int,bigint
 alter table alltypes_test partition(month=cast(1+4 as int), year=cast(100*20+9 as int))
 set location '$FILESYSTEM_PREFIX/test-warehouse/alltypes_rc/year=2009/month=5'
 ---- RESULTS
+'New location has been set for the specified partition.'
 ====
 ---- QUERY
 alter table alltypes_test partition(month=cast(2+3 as int), year=2009)
@@ -661,6 +684,7 @@ STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 # IMPALA-1016: Testing scanning newly added columns
 CREATE TABLE imp1016 (string1 string)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 INSERT INTO imp1016 VALUES ('test')
@@ -670,6 +694,7 @@ INSERT INTO imp1016 VALUES ('test')
 ---- QUERY
 ALTER TABLE imp1016 ADD COLUMNS (string2 string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 DESCRIBE imp1016
@@ -718,6 +743,7 @@ bigint
 # Create a larger table to test scanning newly added columns
 CREATE TABLE imp1016Large (string1 string)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # There is a codepath that operates on chunks of 1024 tuples, inserting
@@ -729,6 +755,7 @@ INSERT INTO imp1016Large SELECT 'test' FROM functional.alltypes LIMIT 2000
 ---- QUERY
 ALTER TABLE imp1016Large ADD COLUMNS (string2 string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 DESCRIBE imp1016Large
@@ -801,6 +828,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 drop table $DATABASE2.mv2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables in $DATABASE2 like '*mv*'
@@ -855,6 +883,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 drop table $DATABASE2.mv2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables in $DATABASE2 like '*mv*'

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test b/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
index f6dc4c4..6840951 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table test_char_tmp (c char(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_char_tmp select cast("hello" as char(5))
@@ -64,6 +65,7 @@ string
 ---- QUERY
 create table test_varchar_tmp (vc varchar(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_varchar_tmp values (cast("hello" as varchar(5)))
@@ -101,6 +103,7 @@ string
 create table allchars
 (cshort char(5), clong char(140), vc varchar(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allchars values (cast("123456" as char(5)), cast("123456" as char(140)),
@@ -117,6 +120,7 @@ char,char,string
 create table allchars_par
 (cshort char(5), clong char(140), vc varchar(5)) stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allchars_par values (cast("123456" as char(5)), cast("123456" as char(140)),
@@ -193,6 +197,7 @@ test_char_nulls ( c20 char(20),
                   c120 char(120),
                   c140 char(140))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_char_nulls

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
index ddc6da3..50eb24d 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
@@ -609,6 +609,7 @@ STRING
 # The table was not changed. Validate that the next compute incremental stats is a no-op.
 compute incremental stats complextypestbl_part;
 ---- RESULTS
+'No partitions selected for incremental stats update.'
 ---- ERRORS
 No partitions selected for incremental stats update
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
index b7494f0..ce147e4 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
@@ -70,6 +70,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Adding a column shouldn't cause the stats to be dropped.
 alter table alltypes add columns (new_col int)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -99,6 +100,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # HIVE-15075 is resolved.
 alter table alltypes change new_col new_col2 int
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -126,6 +128,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Removing a column shouldn't cause the stats to be dropped.
 alter table alltypes drop column new_col2
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -345,6 +348,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Add partitions with NULL values and check for stats.
 alter table alltypes add partition (year=NULL, month=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -370,6 +374,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 alter table alltypes add partition (year=2011, month=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -1025,9 +1030,13 @@ STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 insert into table empty_partitioned partition (j=2) select 1;
 ====
 ---- QUERY
+drop stats empty_partitioned
+---- RESULTS
+'Stats have been dropped.'
+====
+---- QUERY
 # Verify partition stats work with empty and non-empty partition.
-drop stats empty_partitioned;
-compute stats empty_partitioned;
+compute stats empty_partitioned
 ---- RESULTS
 'Updated 2 partition(s) and 1 column(s).'
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-database.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-database.test b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
index 1fcf85c..765bca8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-database.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create database $DATABASE_2 comment "For testing"
 ---- RESULTS
+'Database has been created.'
 ====
 ---- QUERY
 show databases like "$DATABASE_2"
@@ -15,11 +16,13 @@ STRING, STRING
 # IF NOT EXISTS is specified.
 create database if not exists $DATABASE_2
 ---- RESULTS
+'Database already exists.'
 ====
 ---- QUERY
 # Test dropping the database.
 drop database $DATABASE_2
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like "$DATABASE_2"
@@ -31,6 +34,7 @@ STRING, STRING
 # Dropping a non-existent databases is ok with IF EXISTS
 drop database if exists $DATABASE_2
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 # Test DROP DATABASE ... CASCADE
@@ -50,6 +54,7 @@ create function if not exists $DATABASE_cascade.f1() returns string
 create aggregate function if not exists $DATABASE_cascade.f2(int, string) RETURNS int
   location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdas.so' UPDATE_FN='TwoArgUpdate'
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 show tables in $DATABASE_cascade
@@ -80,6 +85,7 @@ STRING, STRING, STRING, STRING
 # as the database itself.
 drop database $DATABASE_cascade cascade
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like '$DATABASE_cascade'
@@ -99,6 +105,7 @@ STRING,STRING
 ---- QUERY
 drop database $DATABASE_restrict restrict
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like '$DATABASE_restrict'

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
index bd54f3d..7a80602 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
@@ -3,6 +3,7 @@
 create table $DATABASE.temp_decimal_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/decimal.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_decimal_table
@@ -17,6 +18,7 @@ STRING, STRING, STRING
 create table $DATABASE.like_zipcodes_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/zipcode_incomes.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_zipcodes_file
@@ -33,6 +35,7 @@ STRING, STRING, STRING
 create table $DATABASE.like_alltypestiny_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/alltypestiny.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_alltypestiny_file
@@ -59,6 +62,7 @@ create external table $DATABASE.like_enumtype_file like parquet
 STORED AS PARQUET
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/schemas/enum'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_enumtype_file
@@ -81,6 +85,7 @@ STRING, STRING
 create table if not exists $DATABASE.like_alltypestiny_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/zipcode_incomes.parquet'
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 # Should not have changed since last statement was IF NOT EXISTS.
@@ -103,11 +108,13 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists allcomplextypes_clone
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table allcomplextypes_clone like functional.allcomplextypes
 stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe allcomplextypes_clone
@@ -133,15 +140,18 @@ STRING, STRING, STRING
 ---- QUERY
 drop table allcomplextypes_clone
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table if exists $DATABASE.temp_legacy_table
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table $DATABASE.temp_legacy_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/legacy_nested.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_legacy_table
@@ -162,15 +172,18 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists $DATABASE.temp_legacy_table
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table if exists $DATABASE.temp_modern_table
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table $DATABASE.temp_modern_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/modern_nested.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_modern_table
@@ -191,6 +204,7 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists $DATABASE.temp_modern_table
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 # Test adding sort.columns when creating a table like a parquet file.

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
index 0a4df27..ee16c37 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
@@ -4,6 +4,7 @@
 create table alltypes_test like functional_seq_snap.alltypes
 stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure no data exists for this table
@@ -35,6 +36,7 @@ BIGINT
 # CREATE TABLE LIKE on a view
 create table like_view like functional.view_view
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe like_view
@@ -67,6 +69,7 @@ BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ---- QUERY
 create table like_view_parquet like functional.view_view stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats like_view_parquet
@@ -82,6 +85,7 @@ BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 create external table jointbl_rc_like like functional_rc_gzip.jointbl
 location '$FILESYSTEM_PREFIX/test-warehouse/jointbl_rc_gzip'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # should get some results back
@@ -97,6 +101,7 @@ BIGINT, STRING, INT, INT
 # CREATE TABLE LIKE on unpartitioned table.
 create table jointbl_like like functional.jointbl
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure the new table can be queried and no data exists for this table.
@@ -110,11 +115,13 @@ BIGINT
 # No error is thrown when IF NOT EXISTS is specified and the table already exists.
 create table if not exists jointbl_like like functional.jointbl
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 # IF NOT EXISTS also applies when the src table is the same as the new table.
 create table if not exists jointbl_like like jointbl_like
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 insert overwrite table jointbl_like
@@ -155,6 +162,7 @@ create table no_avro_schema (
 partitioned by (year int, month int)
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe no_avro_schema
@@ -182,6 +190,7 @@ STRING, STRING, STRING
 # Test creating an Avro table without an Avro schema via CREATE TABLE LIKE (IMPALA-1813)
 create table like_no_avro_schema like no_avro_schema stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe like_no_avro_schema
@@ -208,10 +217,12 @@ STRING, STRING, STRING
 ---- QUERY
 drop table like_no_avro_schema
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table no_avro_schema
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 # Test setting sort.columns when using create table like.

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-table.test b/testdata/workloads/functional-query/queries/QueryTest/create-table.test
index 2661394..7f91adc 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-table.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table $DATABASE.testtbl(i int, s string COMMENT 'String col') STORED AS TEXTFILE
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure creating a table with the same name doesn't throw an error when
@@ -13,6 +14,7 @@ ESCAPED BY '\\'
 LINES TERMINATED BY '\n'
 STORED AS TEXTFILE
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 show tables in $DATABASE
@@ -54,6 +56,7 @@ INT, STRING
 ---- QUERY
 create table $DATABASE.testtbl_part(i int, s string) PARTITIONED BY (id int comment 'C')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Partition columns are displayed as part of DESCRIBE <table>
@@ -105,6 +108,7 @@ STRING
 # Make sure we create the table in the proper database after a "use"
 create table testtbl2(f float, d double) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show tables
@@ -118,6 +122,7 @@ STRING
 ---- QUERY
 drop table testtbl2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables
@@ -134,6 +139,7 @@ with serdeproperties
 ('avro.schema.url'='$FILESYSTEM_PREFIX/test-warehouse/avro_schemas/functional/alltypes.json')
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe avro_alltypes_nopart
@@ -160,6 +166,7 @@ with serdeproperties
 ('avro.schema.url'='$FILESYSTEM_PREFIX/test-warehouse/avro_schemas/functional/alltypes.json')
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe avro_alltypes_part
@@ -183,18 +190,22 @@ STRING, STRING, STRING
 ---- QUERY
 drop table avro_alltypes_part
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table avro_alltypes_nopart
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table testtbl
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table testtbl_part
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables
@@ -205,6 +216,7 @@ STRING
 ---- QUERY
 drop table if exists non_existent_db.tbl
 ---- RESULTS
+'Database does not exist: non_existent_db'
 ====
 ---- QUERY
 # Test table creation with tblproperty and serdeproperty lengths just within limits
@@ -219,17 +231,20 @@ with serdeproperties(
 'valuevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluev
 aluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalueval
 uevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalue
 valuevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalueva
 luevaluevalue'
 )
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
 create table skip_header_test_a (i1 integer) tblproperties('skip.header.line.count'='2')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty on a Parquet table
 create table skip_header_test_d (i1 integer) stored as parquet
 tblproperties('skip.header.line.count'='2')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting an invalid skip.header.line.count tblproperty

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test b/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
index 460aaac..004feff 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
@@ -18,6 +18,7 @@ STRING,STRING,INT,INT
 # create new tables like the ones above to test inserting
 create table tecn like functional.text_thorn_ecirc_newline;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # insert data into tecn table and check results
@@ -35,4 +36,4 @@ select * from tecn
 'efg','xyz',3,4
 ---- TYPES
 STRING,STRING,INT,INT
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test b/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
index 188d54d..9dde519 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
@@ -34,6 +34,7 @@ STRING,STRING,INT,INT
 create table cbn like functional.text_comma_backslash_newline;
 create table dhp like functional.text_dollar_hash_pipe;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # insert data into cbn table and check results
@@ -70,4 +71,4 @@ select * from dhp
 'abc #$#$ abc','xyz $#$# xyz',5,6
 ---- TYPES
 STRING,STRING,INT,INT
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/describe-path.test b/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
index 2547267..d6f041c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
@@ -90,6 +90,7 @@ create table if not exists nested_structs (
   map_array_map_struct_col
   map<string, array<map<string, struct<f1:string, f2:int>>>>)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe nested_structs

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
index d41d9da..a097c61 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
@@ -3,12 +3,21 @@
 # Add functions and test function overloading and scoping.
 create function $DATABASE.fn() RETURNS int
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
+---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function $DATABASE.fn(int) RETURNS double
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
 ====
 ---- QUERY
+# Test IF NOT EXISTS
+create function if not exists $DATABASE.fn(int) RETURNS double
+LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
+---- RESULTS
+'Function already exists.'
+====
+---- QUERY
 create function $DATABASE.fn(int, string) RETURNS int
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
 ====
@@ -149,6 +158,8 @@ STRING
 ====
 ---- QUERY
 drop function $DATABASE.fn2(int, string)
+---- RESULTS
+'Function has been dropped.'
 ====
 ---- QUERY
 show functions
@@ -199,6 +210,11 @@ STRING, STRING, STRING, STRING
 drop function fn()
 ====
 ---- QUERY
+drop function if exists fn()
+---- RESULTS
+'Function does not exist.'
+====
+---- QUERY
 show functions;
 ---- LABELS
 return type, signature, binary type, is persistent

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
index a69a93f..e7f8f2d 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
@@ -2,18 +2,22 @@
 ---- QUERY
 create role grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_TEST_DB
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_SELECT_INSERT_TEST_TBL
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_URI
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 # Shows all roles in the system
@@ -489,10 +493,12 @@ root
 REVOKE ROLE grant_revoke_test_ALL_URI FROM GROUP `$GROUP_NAME`;
 REVOKE ROLE grant_revoke_test_SELECT_INSERT_TEST_TBL FROM GROUP `$GROUP_NAME`;
 ---- RESULTS
+'Role has been revoked.'
 ====
 ---- QUERY
 GRANT ROLE grant_revoke_test_ALL_SERVER TO GROUP `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 show current roles
@@ -505,10 +511,12 @@ STRING
 # Create a table with multiple columns to test column-level security.
 create table grant_rev_db.test_tbl3(a int, b int, c int, d int, e int) partitioned by (x int, y int)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 GRANT SELECT (a, b, x) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -525,6 +533,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 GRANT SELECT (c, d, y) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -544,6 +553,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 GRANT SELECT (a, a, e, x) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -565,6 +575,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 # Revoke SELECT privileges from columns
 REVOKE SELECT (a, b, b, y) ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -582,6 +593,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 REVOKE SELECT (a, b, c, x) ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -597,6 +609,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 REVOKE SELECT (a, b, c, d, e) ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ALL_SERVER;
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -613,6 +626,7 @@ GRANT ROLE grant_revoke_test_ROOT TO GROUP root;
 GRANT SELECT ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT;
 REVOKE ALL ON DATABASE functional FROM grant_revoke_test_ROOT;
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- USER
 root
@@ -635,17 +649,20 @@ User 'root' does not have privileges to execute: GRANT_PRIVILEGE
 ---- QUERY
 REVOKE SELECT ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ROOT
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 # Grant SELECT on table to 'root' with 'WITH GRANT' option.
 GRANT SELECT ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT WITH GRANT OPTION
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- USER
 root
 ---- QUERY
 GRANT SELECT (a) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- USER
 root
@@ -662,6 +679,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 GRANT SELECT (a, c, e) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ALL_SERVER  WITH GRANT OPTION
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -678,6 +696,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 ---- QUERY
 REVOKE GRANT OPTION FOR SELECT (a, c) ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 # TODO: Add a test case that exercises the cascading effect of REVOKE ALL.
@@ -699,6 +718,7 @@ revoke role grant_revoke_test_ALL_SERVER from group `$GROUP_NAME`
 # Test 'grant all on server' with explicit server name specified.
 create role grant_revoke_test_ALL_SERVER1
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 grant all on server server1 to grant_revoke_test_ALL_SERVER1
@@ -745,6 +765,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, STRING
 # to a table in the database
 grant role grant_revoke_test_ALL_SERVER to group `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 create role grant_revoke_test_COLUMN_PRIV
@@ -800,12 +821,15 @@ STRING,STRING
 ---- QUERY
 grant role grant_revoke_test_ALL_SERVER to group `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 drop database if exists grant_rev_db cascade
 ====
 ---- QUERY
 revoke role grant_revoke_test_ALL_SERVER from group `$GROUP_NAME`
+---- RESULTS
+'Role has been revoked.'
 ====
 ---- QUERY
 revoke role grant_revoke_test_COLUMN_PRIV from group `$GROUP_NAME`
@@ -818,4 +842,5 @@ drop role grant_revoke_test_ALL_URI;
 drop role grant_revoke_test_ROOT;
 drop role grant_revoke_test_COLUMN_PRIV;
 ---- RESULTS
+'Role has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
index a3b9354..f51075c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
@@ -2,10 +2,12 @@
 ---- QUERY
 create role grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_TEST_DB
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 show roles
@@ -156,6 +158,7 @@ does not have privileges to access: grant_rev_db.kudu_tbl
 ---- QUERY
 grant select(a) on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 grant ALL on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
@@ -184,4 +187,5 @@ drop role grant_revoke_test_ALL_SERVER;
 drop role grant_revoke_test_ALL_TEST_DB;
 drop role grant_revoke_test_KUDU;
 ---- RESULTS
+'Role has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test b/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
index a217de7..a0ecb18 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table alltypessmall_hbase like functional_hbase.alltypessmall
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 compute incremental stats alltypessmall_hbase

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test b/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
index 9d6b9d1..43edb36 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
@@ -6,6 +6,7 @@ create function if not exists twenty_one_args(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='TwentyOneArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 # Regression test for IMPALA-6262: failure to initialize the output expressions
@@ -21,5 +22,6 @@ Cannot interpret native UDF 'twenty_one_args': number of arguments is more than
 drop function twenty_one_args(int, int, int, int, int, int, int, int,
     int, int, int, int, int, int, int, int, int, int, int, int, int);
 ---- RESULTS
+'Function has been dropped.'
 ====
 

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test b/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
index dfdb2ed..696644f 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
@@ -3,6 +3,7 @@
 create database insert_permutation_test location
 '$FILESYSTEM_PREFIX/test-warehouse/insert_permutation_test'
 ---- RESULTS
+'Database has been created.'
 ====
 ---- QUERY
 use insert_permutation_test
@@ -14,6 +15,7 @@ create table perm_part(int_col1 int, string_col string) partitioned by (p1 int,
 create table parquet_part(int_col1 int, string_col string)
 partitioned by (p1 int, p2 string) stored as parquet;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Simple non-permutation

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
index 305ccf1..85937fa 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
@@ -3,6 +3,7 @@
 create table simple (id int primary key, name string, valf float, vali bigint)
   partition by hash (id) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Hash partitions cannot be enumerated as range partitions
@@ -44,6 +45,7 @@ ImpalaRuntimeException: Kudu table 'impala::$DATABASE.simple' does not exist on
 ---- QUERY
 alter table simple rename to simple_new;
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 select count(*) from simple_new;
@@ -57,6 +59,7 @@ BIGINT
 create table tbl_to_alter (id int primary key, name string null, vali bigint not null)
   partition by range (id) (partition 1 < values <= 10) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -68,6 +71,7 @@ show range partitions tbl_to_alter;
 # Add a range partition
 alter table tbl_to_alter add range partition 10 < values <= 20
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -93,6 +97,7 @@ INT,STRING,BIGINT
 # Add a singleton range partition
 alter table tbl_to_alter add range partition value = 100
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -120,6 +125,7 @@ INT,STRING,BIGINT
 # Add an unbounded range partition
 alter table tbl_to_alter add range partition 1000 < values
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -141,11 +147,13 @@ NonRecoverableException: New range partition conflicts with existing range parti
 # to hide the error
 alter table tbl_to_alter add if not exists range partition 10 < values <= 30
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Drop one of the recently inserted partitions
 alter table tbl_to_alter drop range partition value = 100
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -167,6 +175,7 @@ INT,STRING,BIGINT
 # Drop an existing range partition
 alter table tbl_to_alter drop range partition 11 <= values < 21
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -180,6 +189,7 @@ show range partitions tbl_to_alter;
 alter table tbl_to_alter drop range partition 1 < values <= 10;
 alter table tbl_to_alter drop range partition 1000 < values
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -209,6 +219,7 @@ alter table tbl_to_alter add range partition 1 < values <= 20;
 alter table tbl_to_alter add columns (new_col1 int not null default 10,
   new_col2 bigint not null default 1000)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -261,6 +272,7 @@ INT,STRING,BIGINT,INT,BIGINT
 # Add nullable columns: with and without a default
 alter table tbl_to_alter add columns (new_col3 string null, new_col4 int null default -1)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 # Add a row
@@ -307,6 +319,7 @@ A new non-null column must have a default value
 # Drop a column
 alter table tbl_to_alter drop column vali
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 # Retrieve table rows after column got dropped
@@ -330,6 +343,7 @@ NonRecoverableException: cannot remove a key column
 # Rename a column
 alter table tbl_to_alter change column new_col3 last_name string
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 # Ensure the renamed column is accessible
@@ -355,6 +369,7 @@ BIGINT
 # Rename the Impala table
 alter table tbl_to_alter rename to kudu_tbl_to_alter
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Ensure the Impala table is accessible after it got renamed
@@ -421,6 +436,7 @@ partition by range (
             cast('2009-01-02 00:00:00' as timestamp)
 ) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -434,6 +450,7 @@ alter table ts_ranges add range partition
 cast('2009-01-02 00:00:00' as timestamp) <= VALUES <
 cast('2009-01-03 00:00:00' as timestamp)
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -448,6 +465,7 @@ alter table ts_ranges drop range partition
 cast('2009-01-02 00:00:00' as timestamp) <= VALUES <
 cast('2009-01-03 00:00:00' as timestamp)
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 show range partitions ts_ranges


[6/6] impala git commit: IMPALA-5717: Support for reading ORC data files

Posted by ta...@apache.org.
IMPALA-5717: Support for reading ORC data files

This patch integrates the orc library into Impala and implements
HdfsOrcScanner as a middle layer between them. The HdfsOrcScanner
supplies input needed from the orc-reader, tracks memory consumption of
the reader and transfers the reader's output (orc::ColumnVectorBatch)
into impala::RowBatch. The ORC version we used is release-1.4.3.

A startup option --enable_orc_scanner is added for this feature. It's
set to true by default. Setting it to false will fail queries on ORC
tables.

Currently, we only support reading primitive types. Writing into ORC
table has not been supported neither.

Tests
 - Most of the end-to-end tests can run on ORC format.
 - Add tpcds, tpch tests for ORC.
 - Add some ORC specific tests.
 - Haven't enabled test_scanner_fuzz for ORC yet, since the ORC library
   is not robust for corrupt files (ORC-315).

Change-Id: Ia7b6ae4ce3b9ee8125b21993702faa87537790a4
Reviewed-on: http://gerrit.cloudera.org:8080/9134
Reviewed-by: Quanlong Huang <hu...@gmail.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 818cd8fa2721cd8205b304563b728952bffc8b2f
Parents: d28b39a
Author: stiga-huang <hu...@gmail.com>
Authored: Thu Jan 25 06:39:25 2018 -0800
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Apr 11 05:13:02 2018 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  |   5 +
 be/CMakeLists.txt                               |   2 +
 be/src/codegen/gen_ir_descriptions.py           |   4 +-
 be/src/exec/CMakeLists.txt                      |   1 +
 be/src/exec/hdfs-orc-scanner.cc                 | 763 +++++++++++++++++++
 be/src/exec/hdfs-orc-scanner.h                  | 224 ++++++
 be/src/exec/hdfs-parquet-scanner-ir.cc          |  14 -
 be/src/exec/hdfs-parquet-scanner.cc             | 185 +----
 be/src/exec/hdfs-parquet-scanner.h              |  65 --
 be/src/exec/hdfs-scan-node-base.cc              |   6 +
 be/src/exec/hdfs-scan-node-mt.cc                |   1 +
 be/src/exec/hdfs-scanner-ir.cc                  |  14 +
 be/src/exec/hdfs-scanner.cc                     | 179 +++++
 be/src/exec/hdfs-scanner.h                      |  76 ++
 be/src/util/backend-gflag-util.cc               |   2 +
 bin/bootstrap_toolchain.py                      |   2 +-
 bin/impala-config.sh                            |   2 +
 cmake_modules/FindOrc.cmake                     |  55 ++
 common/thrift/BackendGflags.thrift              |   2 +
 common/thrift/CatalogObjects.thrift             |   6 +-
 fe/src/main/cup/sql-parser.cup                  |   8 +-
 .../impala/analysis/ComputeStatsStmt.java       |   8 +-
 .../apache/impala/catalog/HdfsFileFormat.java   |  38 +-
 .../impala/catalog/HdfsStorageDescriptor.java   |   1 +
 .../org/apache/impala/planner/HdfsScanNode.java |   8 +-
 .../apache/impala/service/BackendConfig.java    |   2 +
 .../org/apache/impala/service/Frontend.java     |   4 +-
 fe/src/main/jflex/sql-scanner.flex              |   1 +
 testdata/LineItemMultiBlock/README.dox          |  19 +
 .../lineitem_orc_multiblock_one_stripe.orc      | Bin 0 -> 424277 bytes
 .../LineItemMultiBlock/lineitem_sixblocks.orc   | Bin 0 -> 863581 bytes
 .../LineItemMultiBlock/lineitem_threeblocks.orc | Bin 0 -> 465929 bytes
 testdata/bin/create-load-data.sh                |   3 +
 testdata/bin/generate-schema-statements.py      |   7 +-
 testdata/bin/run-hive-server.sh                 |   5 +-
 .../common/etc/hadoop/conf/hdfs-site.xml.tmpl   |   6 +
 testdata/data/chars-formats.orc                 | Bin 0 -> 1411 bytes
 .../functional/functional_schema_template.sql   |   2 +
 .../datasets/functional/schema_constraints.csv  |   3 +
 .../PlannerTest/complex-types-file-formats.test |  57 +-
 .../functional-query/functional-query_core.csv  |   1 +
 .../functional-query_dimensions.csv             |   2 +-
 .../functional-query_exhaustive.csv             |   1 +
 .../functional-query_pairwise.csv               |   1 +
 .../queries/DataErrorsTest/orc-type-checks.test | 127 +++
 testdata/workloads/tpcds/tpcds_core.csv         |   1 +
 testdata/workloads/tpcds/tpcds_dimensions.csv   |   2 +-
 testdata/workloads/tpcds/tpcds_exhaustive.csv   |   3 +
 testdata/workloads/tpcds/tpcds_pairwise.csv     |   3 +
 testdata/workloads/tpch/tpch_core.csv           |   1 +
 testdata/workloads/tpch/tpch_dimensions.csv     |   2 +-
 testdata/workloads/tpch/tpch_exhaustive.csv     |   3 +
 testdata/workloads/tpch/tpch_pairwise.csv       |   3 +
 tests/common/impala_test_suite.py               |   2 +-
 tests/common/test_dimensions.py                 |   2 +-
 tests/common/test_vector.py                     |   2 +-
 tests/comparison/cli_options.py                 |   2 +-
 tests/query_test/test_chars.py                  |   6 +
 tests/query_test/test_decimal_queries.py        |   2 +-
 tests/query_test/test_scanners.py               | 102 ++-
 tests/query_test/test_scanners_fuzz.py          |   2 +
 tests/query_test/test_tpch_queries.py           |   2 +-
 62 files changed, 1745 insertions(+), 307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 612e00c..43cd258 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -76,6 +76,7 @@ endfunction()
 # set_dep_root(PACKAGE) ->
 #   PACKAGE_ROOT set to $ENV{IMPALA_TOOLCHAIN}/PACKAGE-$ENV{IMPALA_PACKAGE_VERSION}
 set_dep_root(AVRO)
+set_dep_root(ORC)
 set_dep_root(BOOST)
 set_dep_root(BREAKPAD)
 set_dep_root(BZIP2)
@@ -272,6 +273,10 @@ message(STATUS "RapidJson include dir: " ${RAPIDJSON_INCLUDE_DIR})
 find_package(Avro REQUIRED)
 IMPALA_ADD_THIRDPARTY_LIB(avro ${AVRO_INCLUDE_DIR} ${AVRO_STATIC_LIB} "")
 
+# find ORC headers and libs
+find_package(Orc REQUIRED)
+IMPALA_ADD_THIRDPARTY_LIB(orc ${ORC_INCLUDE_DIR} ${ORC_STATIC_LIB} "")
+
 # find protobuf headers, libs and compiler
 find_package(Protobuf REQUIRED)
 IMPALA_ADD_THIRDPARTY_LIB(protobuf ${PROTOBUF_INCLUDE_DIR} ${PROTOBUF_STATIC_LIBRARY}

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index b6e10b0..8e4f8bd 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -295,6 +295,7 @@ set(CLANG_INCLUDE_FLAGS
   "-I${GFLAGS_INCLUDE_DIR}"
   "-I${RAPIDJSON_INCLUDE_DIR}"
   "-I${AVRO_INCLUDE_DIR}"
+  "-I${ORC_INCLUDE_DIR}"
   # Include Boost as a system directory to suppress warnings from headers.
   "-isystem${BOOST_INCLUDEDIR}"
   # Required so that jni.h can be found during Clang compilation
@@ -447,6 +448,7 @@ set (IMPALA_DEPENDENCIES
   zlib
   bzip2
   avro
+  orc
   java_jvm
   kudu_client)
 

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/codegen/gen_ir_descriptions.py
----------------------------------------------------------------------
diff --git a/be/src/codegen/gen_ir_descriptions.py b/be/src/codegen/gen_ir_descriptions.py
index 1d0f38e..26a8ad7 100755
--- a/be/src/codegen/gen_ir_descriptions.py
+++ b/be/src/codegen/gen_ir_descriptions.py
@@ -177,8 +177,8 @@ ir_functions = [
    "_ZN6impala11HdfsScanner18WriteAlignedTuplesEPNS_7MemPoolEPNS_8TupleRowEPNS_13FieldLocationEiiiib"],
   ["PROCESS_SCRATCH_BATCH",
    "_ZN6impala18HdfsParquetScanner19ProcessScratchBatchEPNS_8RowBatchE"],
-  ["PARQUET_SCANNER_EVAL_RUNTIME_FILTER",
-   "_ZN6impala18HdfsParquetScanner17EvalRuntimeFilterEiPNS_8TupleRowE"],
+  ["HDFS_SCANNER_EVAL_RUNTIME_FILTER",
+   "_ZN6impala11HdfsScanner17EvalRuntimeFilterEiPNS_8TupleRowE"],
   ["STRING_TO_BOOL", "IrStringToBool"],
   ["STRING_TO_INT8", "IrStringToInt8"],
   ["STRING_TO_INT16", "IrStringToInt16"],

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index ddd84ee..7224df8 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -59,6 +59,7 @@ add_library(Exec
   hdfs-parquet-scanner.cc
   hdfs-parquet-scanner-ir.cc
   hdfs-parquet-table-writer.cc
+  hdfs-orc-scanner.cc
   hbase-scan-node.cc
   hbase-table-scanner.cc
   incr-stats-util.cc

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-orc-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-orc-scanner.cc b/be/src/exec/hdfs-orc-scanner.cc
new file mode 100644
index 0000000..3660600
--- /dev/null
+++ b/be/src/exec/hdfs-orc-scanner.cc
@@ -0,0 +1,763 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/hdfs-orc-scanner.h"
+
+#include <queue>
+
+#include "exec/scanner-context.inline.h"
+#include "exprs/expr.h"
+#include "runtime/runtime-filter.inline.h"
+#include "runtime/tuple-row.h"
+#include "util/decompress.h"
+
+#include "common/names.h"
+
+using namespace impala;
+using namespace impala::io;
+
+DEFINE_bool(enable_orc_scanner, true,
+    "If false, reading from ORC format tables is not supported");
+
+Status HdfsOrcScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
+    const vector<HdfsFileDesc*>& files) {
+  for (HdfsFileDesc* file : files) {
+    // If the file size is less than 10 bytes, it is an invalid ORC file.
+    if (file->file_length < 10) {
+      return Status(Substitute("ORC file $0 has an invalid file length: $1",
+          file->filename, file->file_length));
+    }
+  }
+  return IssueFooterRanges(scan_node, THdfsFileFormat::ORC, files);
+}
+
+namespace impala {
+
+HdfsOrcScanner::OrcMemPool::OrcMemPool(HdfsOrcScanner* scanner)
+    : scanner_(scanner), mem_tracker_(scanner_->scan_node_->mem_tracker()) {
+}
+
+HdfsOrcScanner::OrcMemPool::~OrcMemPool() {
+  FreeAll();
+}
+
+void HdfsOrcScanner::OrcMemPool::FreeAll() {
+  int64_t total_bytes_released = 0;
+  for (auto it = chunk_sizes_.begin(); it != chunk_sizes_.end(); ++it) {
+    std::free(it->first);
+    total_bytes_released += it->second;
+  }
+  mem_tracker_->Release(total_bytes_released);
+  chunk_sizes_.clear();
+  ImpaladMetrics::MEM_POOL_TOTAL_BYTES->Increment(-total_bytes_released);
+}
+
+// orc-reader will not check the malloc result. We throw an exception if we can't
+// malloc to stop the orc-reader.
+char* HdfsOrcScanner::OrcMemPool::malloc(uint64_t size) {
+  if (!mem_tracker_->TryConsume(size)) {
+    throw ResourceError(mem_tracker_->MemLimitExceeded(
+        scanner_->state_, "Failed to allocate memory required by ORC library", size));
+  }
+  char* addr = static_cast<char*>(std::malloc(size));
+  if (addr == nullptr) {
+    mem_tracker_->Release(size);
+    throw ResourceError(Status(TErrorCode::MEM_ALLOC_FAILED, size));
+  }
+  chunk_sizes_[addr] = size;
+  ImpaladMetrics::MEM_POOL_TOTAL_BYTES->Increment(size);
+  return addr;
+}
+
+void HdfsOrcScanner::OrcMemPool::free(char* p) {
+  DCHECK(chunk_sizes_.find(p) != chunk_sizes_.end()) << "invalid free!" << endl
+       << GetStackTrace();
+  std::free(p);
+  int64_t size = chunk_sizes_[p];
+  mem_tracker_->Release(size);
+  ImpaladMetrics::MEM_POOL_TOTAL_BYTES->Increment(-size);
+  chunk_sizes_.erase(p);
+}
+
+// TODO: improve this to use async IO (IMPALA-6636).
+void HdfsOrcScanner::ScanRangeInputStream::read(void* buf, uint64_t length,
+    uint64_t offset) {
+  const ScanRange* metadata_range = scanner_->metadata_range_;
+  const ScanRange* split_range =
+      reinterpret_cast<ScanRangeMetadata*>(metadata_range->meta_data())->original_split;
+  int64_t partition_id = scanner_->context_->partition_descriptor()->id();
+
+  // Set expected_local to false to avoid cache on stale data (IMPALA-6830)
+  bool expected_local = false;
+  ScanRange* range = scanner_->scan_node_->AllocateScanRange(
+      metadata_range->fs(), scanner_->filename(), length, offset, partition_id,
+      split_range->disk_id(), expected_local,
+      BufferOpts::ReadInto(reinterpret_cast<uint8_t*>(buf), length));
+
+  unique_ptr<BufferDescriptor> io_buffer;
+  Status status;
+  {
+    SCOPED_TIMER(scanner_->state_->total_storage_wait_timer());
+    status = scanner_->state_->io_mgr()->Read(
+        scanner_->scan_node_->reader_context(), range, &io_buffer);
+  }
+  if (io_buffer != nullptr) scanner_->state_->io_mgr()->ReturnBuffer(move(io_buffer));
+  if (!status.ok()) throw ResourceError(status);
+}
+
+HdfsOrcScanner::HdfsOrcScanner(HdfsScanNodeBase* scan_node, RuntimeState* state)
+  : HdfsScanner(scan_node, state),
+    assemble_rows_timer_(scan_node_->materialize_tuple_timer()) {
+  assemble_rows_timer_.Stop();
+}
+
+HdfsOrcScanner::~HdfsOrcScanner() {
+}
+
+Status HdfsOrcScanner::Open(ScannerContext* context) {
+  RETURN_IF_ERROR(HdfsScanner::Open(context));
+  metadata_range_ = stream_->scan_range();
+  num_cols_counter_ =
+      ADD_COUNTER(scan_node_->runtime_profile(), "NumOrcColumns", TUnit::UNIT);
+  num_stripes_counter_ =
+      ADD_COUNTER(scan_node_->runtime_profile(), "NumOrcStripes", TUnit::UNIT);
+  num_scanners_with_no_reads_counter_ =
+      ADD_COUNTER(scan_node_->runtime_profile(), "NumScannersWithNoReads", TUnit::UNIT);
+  process_footer_timer_stats_ =
+      ADD_SUMMARY_STATS_TIMER(scan_node_->runtime_profile(), "OrcFooterProcessingTime");
+  scan_node_->IncNumScannersCodegenDisabled();
+
+  DCHECK(parse_status_.ok()) << "Invalid parse_status_" << parse_status_.GetDetail();
+  for (const FilterContext& ctx : context->filter_ctxs()) {
+    DCHECK(ctx.filter != nullptr);
+    filter_ctxs_.push_back(&ctx);
+  }
+  filter_stats_.resize(filter_ctxs_.size());
+  reader_mem_pool_.reset(new OrcMemPool(this));
+  reader_options_.setMemoryPool(*reader_mem_pool_);
+
+  // Each scan node can process multiple splits. Each split processes the footer once.
+  // We use a timer to measure the time taken to ProcessFileTail() per split and add
+  // this time to the averaged timer.
+  MonotonicStopWatch single_footer_process_timer;
+  single_footer_process_timer.Start();
+  // First process the file metadata in the footer.
+  Status footer_status = ProcessFileTail();
+  single_footer_process_timer.Stop();
+  process_footer_timer_stats_->UpdateCounter(single_footer_process_timer.ElapsedTime());
+
+  // Release I/O buffers immediately to make sure they are cleaned up
+  // in case we return a non-OK status anywhere below.
+  context_->ReleaseCompletedResources(true);
+  RETURN_IF_ERROR(footer_status);
+
+  // Update orc reader options base on the tuple descriptor
+  RETURN_IF_ERROR(SelectColumns(scan_node_->tuple_desc()));
+
+  // Set top-level template tuple.
+  template_tuple_ = template_tuple_map_[scan_node_->tuple_desc()];
+  return Status::OK();
+}
+
+void HdfsOrcScanner::Close(RowBatch* row_batch) {
+  DCHECK(!is_closed_);
+  if (row_batch != nullptr) {
+    context_->ReleaseCompletedResources(true);
+    row_batch->tuple_data_pool()->AcquireData(template_tuple_pool_.get(), false);
+    if (scan_node_->HasRowBatchQueue()) {
+      static_cast<HdfsScanNode*>(scan_node_)->AddMaterializedRowBatch(
+          unique_ptr<RowBatch>(row_batch));
+    }
+  } else {
+    template_tuple_pool_->FreeAll();
+    context_->ReleaseCompletedResources(true);
+  }
+  scratch_batch_.reset(nullptr);
+
+  // Verify all resources (if any) have been transferred.
+  DCHECK_EQ(template_tuple_pool_->total_allocated_bytes(), 0);
+
+  assemble_rows_timer_.Stop();
+  assemble_rows_timer_.ReleaseCounter();
+
+  THdfsCompression::type compression_type = THdfsCompression::NONE;
+  if (reader_ != nullptr) {
+    compression_type = TranslateCompressionKind(reader_->getCompression());
+  }
+  scan_node_->RangeComplete(THdfsFileFormat::ORC, compression_type);
+
+  for (int i = 0; i < filter_ctxs_.size(); ++i) {
+    const FilterStats* stats = filter_ctxs_[i]->stats;
+    const LocalFilterStats& local = filter_stats_[i];
+    stats->IncrCounters(FilterStats::ROWS_KEY, local.total_possible,
+        local.considered, local.rejected);
+  }
+  CloseInternal();
+}
+
+Status HdfsOrcScanner::ProcessFileTail() {
+  unique_ptr<orc::InputStream> input_stream(new ScanRangeInputStream(this));
+  VLOG_FILE << "Processing FileTail of ORC file: " << input_stream->getName()
+      << ", length: " << input_stream->getLength();
+  try {
+    reader_ = orc::createReader(move(input_stream), reader_options_);
+  } catch (ResourceError& e) {  // errors throw from the orc scanner
+    parse_status_ = e.GetStatus();
+    return parse_status_;
+  } catch (std::exception& e) { // other errors throw from the orc library
+    string msg = Substitute("Encountered parse error in tail of ORC file $0: $1",
+        filename(), e.what());
+    parse_status_ = Status(msg);
+    return parse_status_;
+  }
+
+  if (reader_->getNumberOfRows() == 0)  return Status::OK();
+  if (reader_->getNumberOfStripes() == 0) {
+    return Status(Substitute("Invalid ORC file: $0. No stripes in this file but"
+        " numberOfRows in footer is $1", filename(), reader_->getNumberOfRows()));
+  }
+  return Status::OK();
+}
+
+inline THdfsCompression::type HdfsOrcScanner::TranslateCompressionKind(
+    orc::CompressionKind kind) {
+  switch (kind) {
+    case orc::CompressionKind::CompressionKind_NONE: return THdfsCompression::NONE;
+    // zlib used in ORC is corresponding to Deflate in Impala
+    case orc::CompressionKind::CompressionKind_ZLIB: return THdfsCompression::DEFLATE;
+    case orc::CompressionKind::CompressionKind_SNAPPY: return THdfsCompression::SNAPPY;
+    case orc::CompressionKind::CompressionKind_LZO: return THdfsCompression::LZO;
+    case orc::CompressionKind::CompressionKind_LZ4: return THdfsCompression::LZ4;
+    case orc::CompressionKind::CompressionKind_ZSTD: return THdfsCompression::ZSTD;
+    default:
+      VLOG_QUERY << "Unknown compression kind of orc::CompressionKind: " << kind;
+  }
+  return THdfsCompression::DEFAULT;
+}
+
+Status HdfsOrcScanner::SelectColumns(const TupleDescriptor* tuple_desc) {
+  list<uint64_t> selected_indices;
+  int num_columns = 0;
+  const orc::Type& root_type = reader_->getType();
+  // TODO validate columns. e.g. scale of decimal type
+  for (SlotDescriptor* slot_desc: tuple_desc->slots()) {
+    // Skip partition columns
+    if (slot_desc->col_pos() < scan_node_->num_partition_keys()) continue;
+
+    const SchemaPath &path = slot_desc->col_path();
+    DCHECK_EQ(path.size(), 1);
+    int col_idx = path[0];
+    // The first index in a path includes the table's partition keys
+    int col_idx_in_file = col_idx - scan_node_->num_partition_keys();
+    if (col_idx_in_file >= root_type.getSubtypeCount()) {
+      // In this case, we are selecting a column that is not in the file.
+      // Update the template tuple to put a NULL in this slot.
+      Tuple** template_tuple = &template_tuple_map_[tuple_desc];
+      if (*template_tuple == nullptr) {
+        *template_tuple =
+            Tuple::Create(tuple_desc->byte_size(), template_tuple_pool_.get());
+      }
+      (*template_tuple)->SetNull(slot_desc->null_indicator_offset());
+      continue;
+    }
+    selected_indices.push_back(col_idx_in_file);
+    const orc::Type* orc_type = root_type.getSubtype(col_idx_in_file);
+    const ColumnType& col_type = scan_node_->hdfs_table()->col_descs()[col_idx].type();
+    // TODO(IMPALA-6503): Support reading complex types from ORC format files
+    DCHECK(!col_type.IsComplexType()) << "Complex types are not supported yet";
+    RETURN_IF_ERROR(ValidateType(col_type, *orc_type));
+    col_id_slot_map_[orc_type->getColumnId()] = slot_desc;
+    ++num_columns;
+  }
+  COUNTER_SET(num_cols_counter_, static_cast<int64_t>(num_columns));
+  row_reader_options.include(selected_indices);
+  return Status::OK();
+}
+
+Status HdfsOrcScanner::ValidateType(const ColumnType& type, const orc::Type& orc_type) {
+  switch (orc_type.getKind()) {
+    case orc::TypeKind::BOOLEAN:
+      if (type.type == TYPE_BOOLEAN) return Status::OK();
+      break;
+    case orc::TypeKind::BYTE:
+      if (type.type == TYPE_TINYINT || type.type == TYPE_SMALLINT
+          || type.type == TYPE_INT || type.type == TYPE_BIGINT)
+        return Status::OK();
+      break;
+    case orc::TypeKind::SHORT:
+      if (type.type == TYPE_SMALLINT || type.type == TYPE_INT
+          || type.type == TYPE_BIGINT)
+        return Status::OK();
+      break;
+    case orc::TypeKind::INT:
+      if (type.type == TYPE_INT || type.type == TYPE_BIGINT) return Status::OK();
+      break;
+    case orc::TypeKind::LONG:
+      if (type.type == TYPE_BIGINT) return Status::OK();
+      break;
+    case orc::TypeKind::FLOAT:
+    case orc::TypeKind::DOUBLE:
+      if (type.type == TYPE_FLOAT || type.type == TYPE_DOUBLE) return Status::OK();
+      break;
+    case orc::TypeKind::STRING:
+    case orc::TypeKind::VARCHAR:
+    case orc::TypeKind::CHAR:
+      if (type.type == TYPE_STRING || type.type == TYPE_VARCHAR
+          || type.type == TYPE_CHAR)
+        return Status::OK();
+      break;
+    case orc::TypeKind::TIMESTAMP:
+      if (type.type == TYPE_TIMESTAMP) return Status::OK();
+      break;
+    case orc::TypeKind::DECIMAL: {
+      if (type.type != TYPE_DECIMAL || type.scale != orc_type.getScale()) break;
+      bool overflow = false;
+      int orc_precision = orc_type.getPrecision();
+      if (orc_precision == 0 || orc_precision > ColumnType::MAX_DECIMAL8_PRECISION) {
+        // For ORC decimals whose precision is larger than 18, its value can't fit into
+        // an int64 (10^19 > 2^63). So we should use int128 (16 bytes) for this case.
+        // The possible byte sizes for Impala decimals are 4, 8, 16.
+        // We mark it as overflow if the target byte size is not 16.
+        overflow = (type.GetByteSize() != 16);
+      } else if (orc_type.getPrecision() > ColumnType::MAX_DECIMAL4_PRECISION) {
+        // For ORC decimals whose precision <= 18 and > 9, int64 and int128 can fit them.
+        // We only mark it as overflow if the target byte size is 4.
+        overflow = (type.GetByteSize() == 4);
+      }
+      if (!overflow) return Status::OK();
+      return Status(Substitute(
+          "It can't be truncated to table column $2 for column $0 in ORC file '$1'",
+          orc_type.toString(), filename(), type.DebugString()));
+    }
+    default: break;
+  }
+  return Status(Substitute(
+      "Type mismatch: table column $0 is map to column $1 in ORC file '$2'",
+      type.DebugString(), orc_type.toString(), filename()));
+}
+
+Status HdfsOrcScanner::ProcessSplit() {
+  DCHECK(scan_node_->HasRowBatchQueue());
+  HdfsScanNode* scan_node = static_cast<HdfsScanNode*>(scan_node_);
+  do {
+    unique_ptr<RowBatch> batch = make_unique<RowBatch>(scan_node_->row_desc(),
+        state_->batch_size(), scan_node_->mem_tracker());
+    Status status = GetNextInternal(batch.get());
+    // Always add batch to the queue because it may contain data referenced by previously
+    // appended batches.
+    scan_node->AddMaterializedRowBatch(move(batch));
+    RETURN_IF_ERROR(status);
+    ++row_batches_produced_;
+    if ((row_batches_produced_ & (BATCHES_PER_FILTER_SELECTIVITY_CHECK - 1)) == 0) {
+      CheckFiltersEffectiveness();
+    }
+  } while (!eos_ && !scan_node_->ReachedLimit());
+  return Status::OK();
+}
+
+Status HdfsOrcScanner::GetNextInternal(RowBatch* row_batch) {
+  if (scan_node_->IsZeroSlotTableScan()) {
+    uint64_t file_rows = reader_->getNumberOfRows();
+    // There are no materialized slots, e.g. count(*) over the table.  We can serve
+    // this query from just the file metadata.  We don't need to read the column data.
+    if (stripe_rows_read_ == file_rows) {
+      eos_ = true;
+      return Status::OK();
+    }
+    assemble_rows_timer_.Start();
+    DCHECK_LT(stripe_rows_read_, file_rows);
+    int64_t rows_remaining = file_rows - stripe_rows_read_;
+    int max_tuples = min<int64_t>(row_batch->capacity(), rows_remaining);
+    TupleRow* current_row = row_batch->GetRow(row_batch->AddRow());
+    int num_to_commit = WriteTemplateTuples(current_row, max_tuples);
+    Status status = CommitRows(num_to_commit, row_batch);
+    assemble_rows_timer_.Stop();
+    RETURN_IF_ERROR(status);
+    stripe_rows_read_ += max_tuples;
+    COUNTER_ADD(scan_node_->rows_read_counter(), num_to_commit);
+    return Status::OK();
+  }
+
+  // reset tuple memory. We'll allocate it the first time we use it.
+  tuple_mem_ = nullptr;
+  tuple_ = nullptr;
+
+  // Transfer remaining tuples from the scratch batch.
+  if (ScratchBatchNotEmpty()) {
+    assemble_rows_timer_.Start();
+    RETURN_IF_ERROR(TransferScratchTuples(row_batch));
+    assemble_rows_timer_.Stop();
+    if (row_batch->AtCapacity()) return Status::OK();
+    DCHECK_EQ(scratch_batch_tuple_idx_, scratch_batch_->numElements);
+  }
+
+  while (advance_stripe_ || end_of_stripe_) {
+    context_->ReleaseCompletedResources(/* done */ true);
+    // Commit the rows to flush the row batch from the previous stripe
+    RETURN_IF_ERROR(CommitRows(0, row_batch));
+
+    RETURN_IF_ERROR(NextStripe());
+    DCHECK_LE(stripe_idx_, reader_->getNumberOfStripes());
+    if (stripe_idx_ == reader_->getNumberOfStripes()) {
+      eos_ = true;
+      DCHECK(parse_status_.ok());
+      return Status::OK();
+    }
+  }
+
+  // Apply any runtime filters to static tuples containing the partition keys for this
+  // partition. If any filter fails, we return immediately and stop processing this
+  // scan range.
+  if (!scan_node_->PartitionPassesFilters(context_->partition_descriptor()->id(),
+      FilterStats::ROW_GROUPS_KEY, context_->filter_ctxs())) {
+    eos_ = true;
+    DCHECK(parse_status_.ok());
+    return Status::OK();
+  }
+  assemble_rows_timer_.Start();
+  Status status = AssembleRows(row_batch);
+  assemble_rows_timer_.Stop();
+  RETURN_IF_ERROR(status);
+  if (!parse_status_.ok()) {
+    RETURN_IF_ERROR(state_->LogOrReturnError(parse_status_.msg()));
+    parse_status_ = Status::OK();
+  }
+  return Status::OK();
+}
+
+inline bool HdfsOrcScanner::ScratchBatchNotEmpty() {
+  return scratch_batch_ != nullptr
+      && scratch_batch_tuple_idx_ < scratch_batch_->numElements;
+}
+
+inline static bool CheckStripeOverlapsSplit(int64_t stripe_start, int64_t stripe_end,
+    int64_t split_start, int64_t split_end) {
+  return (split_start >= stripe_start && split_start < stripe_end) ||
+      (split_end > stripe_start && split_end <= stripe_end) ||
+      (split_start <= stripe_start && split_end >= stripe_end);
+}
+
+Status HdfsOrcScanner::NextStripe() {
+  const ScanRange* split_range = static_cast<ScanRangeMetadata*>(
+      metadata_range_->meta_data())->original_split;
+  int64_t split_offset = split_range->offset();
+  int64_t split_length = split_range->len();
+
+  bool start_with_first_stripe = stripe_idx_ == -1;
+  bool misaligned_stripe_skipped = false;
+
+  advance_stripe_ = false;
+  stripe_rows_read_ = 0;
+
+  // Loop until we have found a non-empty stripe.
+  while (true) {
+    // Reset the parse status for the next stripe.
+    parse_status_ = Status::OK();
+
+    ++stripe_idx_;
+    if (stripe_idx_ >= reader_->getNumberOfStripes()) {
+      if (start_with_first_stripe && misaligned_stripe_skipped) {
+        // We started with the first stripe and skipped all the stripes because they were
+        // misaligned. The execution flow won't reach this point if there is at least one
+        // non-empty stripe which this scanner can process.
+        COUNTER_ADD(num_scanners_with_no_reads_counter_, 1);
+      }
+      break;
+    }
+    unique_ptr<orc::StripeInformation> stripe = reader_->getStripe(stripe_idx_);
+    // Also check 'footer_.numberOfRows' to make sure 'select count(*)' and 'select *'
+    // behave consistently for corrupt files that have 'footer_.numberOfRows == 0'
+    // but some data in stripe.
+    if (stripe->getNumberOfRows() == 0 || reader_->getNumberOfRows() == 0) continue;
+
+    uint64_t stripe_offset = stripe->getOffset();
+    uint64_t stripe_len = stripe->getIndexLength() + stripe->getDataLength() +
+        stripe->getFooterLength();
+    int64_t stripe_mid_pos = stripe_offset + stripe_len / 2;
+    if (!(stripe_mid_pos >= split_offset &&
+        stripe_mid_pos < split_offset + split_length)) {
+      // Middle pos not in split, this stripe will be handled by a different scanner.
+      // Mark if the stripe overlaps with the split.
+      misaligned_stripe_skipped |= CheckStripeOverlapsSplit(stripe_offset,
+          stripe_offset + stripe_len, split_offset, split_offset + split_length);
+      continue;
+    }
+
+    // TODO: check if this stripe can be skipped by stats. e.g. IMPALA-6505
+
+    COUNTER_ADD(num_stripes_counter_, 1);
+    row_reader_options.range(stripe->getOffset(), stripe_len);
+    try {
+      row_reader_ = reader_->createRowReader(row_reader_options);
+    } catch (ResourceError& e) {  // errors throw from the orc scanner
+      parse_status_ = e.GetStatus();
+      return parse_status_;
+    } catch (std::exception& e) { // errors throw from the orc library
+      VLOG_QUERY << "Error in creating ORC column readers: " << e.what();
+      parse_status_ = Status(
+          Substitute("Error in creating ORC column readers: $0.", e.what()));
+      return parse_status_;
+    }
+    end_of_stripe_ = false;
+    VLOG_ROW << Substitute("Created RowReader for stripe(offset=$0, len=$1) in file $2",
+        stripe->getOffset(), stripe_len, filename());
+    break;
+  }
+
+  DCHECK(parse_status_.ok());
+  return Status::OK();
+}
+
+Status HdfsOrcScanner::AssembleRows(RowBatch* row_batch) {
+  bool continue_execution = !scan_node_->ReachedLimit() && !context_->cancelled();
+  if (!continue_execution)  return Status::CANCELLED;
+
+  scratch_batch_tuple_idx_ = 0;
+  scratch_batch_ = row_reader_->createRowBatch(row_batch->capacity());
+  DCHECK_EQ(scratch_batch_->numElements, 0);
+
+  int64_t num_rows_read = 0;
+  while (continue_execution) {  // one ORC scratch batch (ColumnVectorBatch) in a round
+    if (scratch_batch_tuple_idx_ == scratch_batch_->numElements) {
+      try {
+        if (!row_reader_->next(*scratch_batch_)) {
+          end_of_stripe_ = true;
+          break; // no more data to process
+        }
+      } catch (ResourceError& e) {
+        parse_status_ = e.GetStatus();
+        return parse_status_;
+      } catch (std::exception& e) {
+        VLOG_QUERY << "Encounter parse error: " << e.what();
+        parse_status_ = Status(Substitute("Encounter parse error: $0.", e.what()));
+        eos_ = true;
+        return parse_status_;
+      }
+      if (scratch_batch_->numElements == 0) {
+        RETURN_IF_ERROR(CommitRows(0, row_batch));
+        end_of_stripe_ = true;
+        return Status::OK();
+      }
+      num_rows_read += scratch_batch_->numElements;
+      scratch_batch_tuple_idx_ = 0;
+    }
+
+    RETURN_IF_ERROR(TransferScratchTuples(row_batch));
+    if (row_batch->AtCapacity()) break;
+    continue_execution &= !scan_node_->ReachedLimit() && !context_->cancelled();
+  }
+  stripe_rows_read_ += num_rows_read;
+  COUNTER_ADD(scan_node_->rows_read_counter(), num_rows_read);
+  return Status::OK();
+}
+
+Status HdfsOrcScanner::TransferScratchTuples(RowBatch* dst_batch) {
+  const TupleDescriptor* tuple_desc = scan_node_->tuple_desc();
+
+  ScalarExprEvaluator* const* conjunct_evals = conjunct_evals_->data();
+  int num_conjuncts = conjunct_evals_->size();
+
+  const orc::Type* root_type = &row_reader_->getSelectedType();
+  DCHECK_EQ(root_type->getKind(), orc::TypeKind::STRUCT);
+
+  DCHECK_LT(dst_batch->num_rows(), dst_batch->capacity());
+  if (tuple_ == nullptr) RETURN_IF_ERROR(AllocateTupleMem(dst_batch));
+  int row_id = dst_batch->num_rows();
+  int capacity = dst_batch->capacity();
+  int num_to_commit = 0;
+  TupleRow* row = dst_batch->GetRow(row_id);
+  Tuple* tuple = tuple_;  // tuple_ is updated in CommitRows
+
+  // TODO(IMPALA-6506): codegen the runtime filter + conjunct evaluation loop
+  // TODO: transfer the scratch_batch_ column-by-column for batch, and then evaluate
+  // the predicates in later loop.
+  while (row_id < capacity && ScratchBatchNotEmpty()) {
+    DCHECK_LT((void*)tuple, (void*)tuple_mem_end_);
+    InitTuple(tuple_desc, template_tuple_, tuple);
+    RETURN_IF_ERROR(ReadRow(static_cast<const orc::StructVectorBatch&>(*scratch_batch_),
+        scratch_batch_tuple_idx_++, root_type, tuple, dst_batch));
+    row->SetTuple(scan_node_->tuple_idx(), tuple);
+    if (!EvalRuntimeFilters(row)) continue;
+    if (ExecNode::EvalConjuncts(conjunct_evals, num_conjuncts, row)) {
+      row = next_row(row);
+      tuple = next_tuple(tuple_desc->byte_size(), tuple);
+      ++row_id;
+      ++num_to_commit;
+    }
+  }
+  VLOG_ROW << Substitute("Transfer $0 rows from scratch batch to dst_batch ($1 rows)",
+      num_to_commit, dst_batch->num_rows());
+  return CommitRows(num_to_commit, dst_batch);
+}
+
+Status HdfsOrcScanner::AllocateTupleMem(RowBatch* row_batch) {
+  int64_t tuple_buffer_size;
+  RETURN_IF_ERROR(
+      row_batch->ResizeAndAllocateTupleBuffer(state_, &tuple_buffer_size, &tuple_mem_));
+  tuple_mem_end_ = tuple_mem_ + tuple_buffer_size;
+  tuple_ = reinterpret_cast<Tuple*>(tuple_mem_);
+  DCHECK_GT(row_batch->capacity(), 0);
+  return Status::OK();
+}
+
+inline Status HdfsOrcScanner::ReadRow(const orc::StructVectorBatch& batch, int row_idx,
+    const orc::Type* orc_type, Tuple* tuple, RowBatch* dst_batch) {
+  for (unsigned int c = 0; c < orc_type->getSubtypeCount(); ++c) {
+    orc::ColumnVectorBatch* col_batch = batch.fields[c];
+    const orc::Type* col_type = orc_type->getSubtype(c);
+    const SlotDescriptor* slot_desc = DCHECK_NOTNULL(
+        col_id_slot_map_[col_type->getColumnId()]);
+    if (col_batch->hasNulls && !col_batch->notNull[row_idx]) {
+      tuple->SetNull(slot_desc->null_indicator_offset());
+      continue;
+    }
+    void* slot_val_ptr = tuple->GetSlot(slot_desc->tuple_offset());
+    switch (col_type->getKind()) {
+      case orc::TypeKind::BOOLEAN: {
+        int64_t val = static_cast<const orc::LongVectorBatch*>(col_batch)->
+            data.data()[row_idx];
+        *(reinterpret_cast<bool*>(slot_val_ptr)) = (val != 0);
+        break;
+      }
+      case orc::TypeKind::BYTE:
+      case orc::TypeKind::SHORT:
+      case orc::TypeKind::INT:
+      case orc::TypeKind::LONG: {
+        const orc::LongVectorBatch* long_batch =
+            static_cast<const orc::LongVectorBatch*>(col_batch);
+        int64_t val = long_batch->data.data()[row_idx];
+        switch (slot_desc->type().type) {
+          case TYPE_TINYINT:
+            *(reinterpret_cast<int8_t*>(slot_val_ptr)) = val;
+            break;
+          case TYPE_SMALLINT:
+            *(reinterpret_cast<int16_t*>(slot_val_ptr)) = val;
+            break;
+          case TYPE_INT:
+            *(reinterpret_cast<int32_t*>(slot_val_ptr)) = val;
+            break;
+          case TYPE_BIGINT:
+            *(reinterpret_cast<int64_t*>(slot_val_ptr)) = val;
+            break;
+          default:
+            DCHECK(false) << "Illegal translation from impala type "
+                << slot_desc->DebugString() << " to orc INT";
+        }
+        break;
+      }
+      case orc::TypeKind::FLOAT:
+      case orc::TypeKind::DOUBLE: {
+        double val =
+            static_cast<const orc::DoubleVectorBatch*>(col_batch)->data.data()[row_idx];
+        if (slot_desc->type().type == TYPE_FLOAT) {
+          *(reinterpret_cast<float*>(slot_val_ptr)) = val;
+        } else {
+          DCHECK_EQ(slot_desc->type().type, TYPE_DOUBLE);
+          *(reinterpret_cast<double*>(slot_val_ptr)) = val;
+        }
+        break;
+      }
+      case orc::TypeKind::STRING:
+      case orc::TypeKind::VARCHAR:
+      case orc::TypeKind::CHAR: {
+        auto str_batch = static_cast<const orc::StringVectorBatch*>(col_batch);
+        const char* src_ptr = str_batch->data.data()[row_idx];
+        int64_t src_len = str_batch->length.data()[row_idx];
+        int dst_len = slot_desc->type().len;
+        if (slot_desc->type().type == TYPE_CHAR) {
+          int unpadded_len = min(dst_len, static_cast<int>(src_len));
+          char* dst_char = reinterpret_cast<char*>(slot_val_ptr);
+          memcpy(dst_char, src_ptr, unpadded_len);
+          StringValue::PadWithSpaces(dst_char, dst_len, unpadded_len);
+          break;
+        }
+        StringValue* dst = reinterpret_cast<StringValue*>(slot_val_ptr);
+        if (slot_desc->type().type == TYPE_VARCHAR && src_len > dst_len) {
+          dst->len = dst_len;
+        } else {
+          dst->len = src_len;
+        }
+        // Space in the StringVectorBatch is allocated by reader_mem_pool_. It will be
+        // reused at next batch, so we allocate a new space for this string.
+        uint8_t* buffer = dst_batch->tuple_data_pool()->TryAllocate(dst->len);
+        if (buffer == nullptr) {
+          string details = Substitute("Could not allocate string buffer of $0 bytes "
+              "for ORC file '$1'.", dst->len, filename());
+          return scan_node_->mem_tracker()->MemLimitExceeded(
+              state_, details, dst->len);
+        }
+        dst->ptr = reinterpret_cast<char*>(buffer);
+        memcpy(dst->ptr, src_ptr, dst->len);
+        break;
+      }
+      case orc::TypeKind::TIMESTAMP: {
+        const orc::TimestampVectorBatch* ts_batch =
+            static_cast<const orc::TimestampVectorBatch*>(col_batch);
+        int64_t secs = ts_batch->data.data()[row_idx];
+        int64_t nanos = ts_batch->nanoseconds.data()[row_idx];
+        *reinterpret_cast<TimestampValue*>(slot_val_ptr) =
+            TimestampValue::FromUnixTimeNanos(secs, nanos);
+        break;
+      }
+      case orc::TypeKind::DECIMAL: {
+        // For decimals whose precision is larger than 18, its value can't fit into
+        // an int64 (10^19 > 2^63). So we should use int128 for this case.
+        if (col_type->getPrecision() == 0 || col_type->getPrecision() > 18) {
+          auto int128_batch = static_cast<const orc::Decimal128VectorBatch*>(col_batch);
+          orc::Int128 orc_val = int128_batch->values.data()[row_idx];
+
+          DCHECK_EQ(slot_desc->type().GetByteSize(), 16);
+          int128_t val = orc_val.getHighBits();
+          val <<= 64;
+          val |= orc_val.getLowBits();
+          // Use memcpy to avoid gcc generating unaligned instructions like movaps
+          // for int128_t. They will raise SegmentFault when addresses are not
+          // aligned to 16 bytes.
+          memcpy(slot_val_ptr, &val, sizeof(int128_t));
+        } else {
+          // Reminder: even decimal(1,1) is stored in int64 batch
+          auto int64_batch = static_cast<const orc::Decimal64VectorBatch*>(col_batch);
+          int64_t val = int64_batch->values.data()[row_idx];
+
+          switch (slot_desc->type().GetByteSize()) {
+            case 4:
+              reinterpret_cast<Decimal4Value*>(slot_val_ptr)->value() = val;
+              break;
+            case 8:
+              reinterpret_cast<Decimal8Value*>(slot_val_ptr)->value() = val;
+              break;
+            case 16:
+              reinterpret_cast<Decimal16Value*>(slot_val_ptr)->value() = val;
+              break;
+            default: DCHECK(false) << "invalidate byte size";
+          }
+        }
+        break;
+      }
+      case orc::TypeKind::LIST:
+      case orc::TypeKind::MAP:
+      case orc::TypeKind::STRUCT:
+      case orc::TypeKind::UNION:
+      default:
+        DCHECK(false) << slot_desc->type().DebugString() << " map to ORC column "
+            << col_type->toString();
+    }
+  }
+  return Status::OK();
+}
+
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-orc-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-orc-scanner.h b/be/src/exec/hdfs-orc-scanner.h
new file mode 100644
index 0000000..837d92a
--- /dev/null
+++ b/be/src/exec/hdfs-orc-scanner.h
@@ -0,0 +1,224 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+
+#ifndef IMPALA_EXEC_HDFS_ORC_SCANNER_H
+#define IMPALA_EXEC_HDFS_ORC_SCANNER_H
+
+#include <orc/OrcFile.hh>
+
+#include "runtime/runtime-state.h"
+#include "exec/hdfs-scanner.h"
+#include "exec/hdfs-scan-node.h"
+#include "util/runtime-profile-counters.h"
+
+namespace impala {
+
+struct HdfsFileDesc;
+
+/// This scanner leverage the ORC library to parse ORC files located in HDFS. Data is
+/// transformed into Impala in-memory representation, i.e. Tuples, RowBatches.
+///
+/// For the file format spec, see https://orc.apache.org/docs/spec-intro.html
+class HdfsOrcScanner : public HdfsScanner {
+ public:
+  /// Exception throws from the orc scanner to stop the orc::RowReader. It's used in
+  /// IO errors (e.g. cancellation) or memory errors (e.g. mem_limit exceeded). The
+  /// exact error message will be recorded in parse_status_.
+  class ResourceError : public std::runtime_error {
+   public:
+    explicit ResourceError(const Status& status)
+      : runtime_error(status.msg().msg()), status_(status) {}
+    virtual ~ResourceError() {}
+    Status& GetStatus() { return status_; }
+
+   private:
+    Status status_;
+  };
+
+  class OrcMemPool : public orc::MemoryPool {
+   public:
+    OrcMemPool(HdfsOrcScanner* scanner);
+    virtual ~OrcMemPool();
+
+    char* malloc(uint64_t size) override;
+    void free(char* p) override;
+
+    void FreeAll();
+   private:
+
+    HdfsOrcScanner* scanner_;
+    MemTracker* mem_tracker_;
+    boost::unordered_map<char*, uint64_t> chunk_sizes_;
+  };
+
+  class ScanRangeInputStream : public orc::InputStream {
+   public:
+    ScanRangeInputStream(HdfsOrcScanner* scanner) {
+      this->scanner_ = scanner;
+      this->filename_ = scanner->filename();
+      this->file_desc_ = scanner->scan_node_->GetFileDesc(
+          scanner->context_->partition_descriptor()->id(), filename_);
+    }
+
+    uint64_t getLength() const {
+      return file_desc_->file_length;
+    }
+
+    uint64_t getNaturalReadSize() const {
+      return scanner_->state_->io_mgr()->max_read_buffer_size();
+    }
+
+    void read(void* buf, uint64_t length, uint64_t offset);
+
+    const std::string& getName() const {
+      return filename_;
+    }
+
+  private:
+    HdfsOrcScanner* scanner_;
+    HdfsFileDesc* file_desc_;
+    std::string filename_;
+  };
+
+  HdfsOrcScanner(HdfsScanNodeBase* scan_node, RuntimeState* state);
+  virtual ~HdfsOrcScanner();
+
+  /// Issue just the footer range for each file.  We'll then parse the footer and pick
+  /// out the columns we want.
+  static Status IssueInitialRanges(HdfsScanNodeBase* scan_node,
+      const std::vector<HdfsFileDesc*>& files) WARN_UNUSED_RESULT;
+
+  virtual Status Open(ScannerContext* context) override WARN_UNUSED_RESULT;
+  virtual Status ProcessSplit() override WARN_UNUSED_RESULT;
+  virtual void Close(RowBatch* row_batch) override;
+
+ private:
+  friend class HdfsOrcScannerTest;
+
+  /// Memory guard of the tuple_mem_
+  uint8_t* tuple_mem_end_ = nullptr;
+
+  /// Index of the current stripe being processed. Initialized to -1 which indicates
+  /// that we have not started processing the first stripe yet (GetNext() has not yet
+  /// been called).
+  int32_t stripe_idx_ = -1;
+
+  /// Counts the number of rows processed for the current stripe.
+  int64_t stripe_rows_read_ = 0;
+
+  /// Indicates whether we should advance to the next stripe in the next GetNext().
+  /// Starts out as true to move to the very first stripe.
+  bool advance_stripe_ = true;
+
+  /// Indicates whether we are at the end of a stripe.
+  bool end_of_stripe_ = true;
+
+  /// Number of scratch batches processed so far.
+  int64_t row_batches_produced_ = 0;
+
+  /// Mem pool used in orc readers.
+  boost::scoped_ptr<OrcMemPool> reader_mem_pool_;
+
+  /// orc::Reader's responsibility is to read the footer and metadata from an ORC file.
+  /// It creates orc::RowReader for further materialization. orc::RowReader is used for
+  /// reading rows from the file.
+  std::unique_ptr<orc::Reader> reader_ = nullptr;
+  std::unique_ptr<orc::RowReader> row_reader_ = nullptr;
+
+  /// Orc reader will write slot values into this scratch batch for top-level tuples.
+  /// See AssembleRows().
+  std::unique_ptr<orc::ColumnVectorBatch> scratch_batch_;
+  int scratch_batch_tuple_idx_ = 0;
+
+  /// ReaderOptions used to create orc::Reader.
+  orc::ReaderOptions reader_options_;
+
+  /// RowReaderOptions used to create orc::RowReader.
+  orc::RowReaderOptions row_reader_options;
+
+  /// Column id is the pre order id in orc::Type tree.
+  /// Map from column id to slot descriptor.
+  boost::unordered_map<int, const SlotDescriptor*> col_id_slot_map_;
+
+  /// Scan range for the metadata.
+  const io::ScanRange* metadata_range_ = nullptr;
+
+  /// Timer for materializing rows. This ignores time getting the next buffer.
+  ScopedTimer<MonotonicStopWatch> assemble_rows_timer_;
+
+  /// Average and min/max time spent processing the footer by each split.
+  RuntimeProfile::SummaryStatsCounter* process_footer_timer_stats_ = nullptr;
+
+  /// Number of columns that need to be read.
+  RuntimeProfile::Counter* num_cols_counter_ = nullptr;
+
+  /// Number of stripes that need to be read.
+  RuntimeProfile::Counter* num_stripes_counter_ = nullptr;
+
+  /// Number of scanners that end up doing no reads because their splits don't overlap
+  /// with the midpoint of any stripe in the file.
+  RuntimeProfile::Counter* num_scanners_with_no_reads_counter_ = nullptr;
+
+  const char *filename() const { return metadata_range_->file(); }
+
+  virtual Status GetNextInternal(RowBatch* row_batch) override WARN_UNUSED_RESULT;
+
+  /// Advances 'stripe_idx_' to the next non-empty stripe and initializes
+  /// row_reader_ to scan it.
+  Status NextStripe() WARN_UNUSED_RESULT;
+
+  /// Reads data using orc-reader to materialize instances of 'tuple_desc'.
+  /// Returns a non-OK status if a non-recoverable error was encountered and execution
+  /// of this query should be terminated immediately.
+  Status AssembleRows(RowBatch* row_batch) WARN_UNUSED_RESULT;
+
+  /// Function used by TransferScratchTuples() to read a single row from scratch_batch_
+  /// into 'tuple'.
+  Status ReadRow(const orc::StructVectorBatch& batch, int row_idx,
+      const orc::Type* orc_type, Tuple* tuple, RowBatch* dst_batch) WARN_UNUSED_RESULT;
+
+  /// Evaluates runtime filters and conjuncts (if any) against the tuples in
+  /// 'scratch_batch_', and adds the surviving tuples to the given batch.
+  /// Returns the number of rows that should be committed to the given batch.
+  Status TransferScratchTuples(RowBatch* dst_batch) WARN_UNUSED_RESULT;
+
+  /// Process the file footer and parse file_metadata_.  This should be called with the
+  /// last FOOTER_SIZE bytes in context_.
+  Status ProcessFileTail() WARN_UNUSED_RESULT;
+
+  /// Update reader options used in orc reader by the given tuple descriptor.
+  Status SelectColumns(const TupleDescriptor* tuple_desc) WARN_UNUSED_RESULT;
+
+  /// Validate whether the ColumnType is compatible with the orc type
+  Status ValidateType(const ColumnType& type, const orc::Type& orc_type)
+      WARN_UNUSED_RESULT;
+
+  /// Part of the HdfsScanner interface, not used in Orc.
+  Status InitNewRange() override WARN_UNUSED_RESULT { return Status::OK(); }
+
+  THdfsCompression::type TranslateCompressionKind(orc::CompressionKind kind);
+
+  inline bool ScratchBatchNotEmpty();
+
+  inline Status AllocateTupleMem(RowBatch* row_batch) WARN_UNUSED_RESULT;
+
+};
+
+} // namespace impala
+
+#endif

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-parquet-scanner-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner-ir.cc b/be/src/exec/hdfs-parquet-scanner-ir.cc
index f2355d8..2ba56c4 100644
--- a/be/src/exec/hdfs-parquet-scanner-ir.cc
+++ b/be/src/exec/hdfs-parquet-scanner-ir.cc
@@ -65,17 +65,3 @@ int HdfsParquetScanner::ProcessScratchBatch(RowBatch* dst_batch) {
   scratch_batch_->tuple_idx += (scratch_tuple - scratch_tuple_start) / tuple_size;
   return output_row - output_row_start;
 }
-
-bool HdfsParquetScanner::EvalRuntimeFilter(int i, TupleRow* row) {
-  LocalFilterStats* stats = &filter_stats_[i];
-  const FilterContext* ctx = filter_ctxs_[i];
-  ++stats->total_possible;
-  if (stats->enabled && ctx->filter->HasFilter()) {
-    ++stats->considered;
-    if (!ctx->Eval(row)) {
-      ++stats->rejected;
-      return false;
-    }
-  }
-  return true;
-}

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 0d79f53..ae22149 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -38,15 +38,6 @@ using std::move;
 using namespace impala;
 using namespace impala::io;
 
-DEFINE_double(parquet_min_filter_reject_ratio, 0.1, "(Advanced) If the percentage of "
-    "rows rejected by a runtime filter drops below this value, the filter is disabled.");
-
-// The number of row batches between checks to see if a filter is effective, and
-// should be disabled. Must be a power of two.
-constexpr int BATCHES_PER_FILTER_SELECTIVITY_CHECK = 16;
-static_assert(BitUtil::IsPowerOf2(BATCHES_PER_FILTER_SELECTIVITY_CHECK),
-    "BATCHES_PER_FILTER_SELECTIVITY_CHECK must be a power of two");
-
 // Max dictionary page header size in bytes. This is an estimate and only needs to be an
 // upper bound.
 const int MAX_DICT_HEADER_SIZE = 100;
@@ -57,7 +48,6 @@ const int MAX_DICT_HEADER_SIZE = 100;
 // THIS RECORDS INFORMATION ABOUT PAST BEHAVIOR. DO NOT CHANGE THIS CONSTANT.
 const int LEGACY_IMPALA_MAX_DICT_ENTRIES = 40000;
 
-const int64_t HdfsParquetScanner::FOOTER_SIZE;
 const int16_t HdfsParquetScanner::ROW_GROUP_END;
 const int16_t HdfsParquetScanner::INVALID_LEVEL;
 const int16_t HdfsParquetScanner::INVALID_POS;
@@ -69,71 +59,14 @@ const string PARQUET_MEM_LIMIT_EXCEEDED =
 
 Status HdfsParquetScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
     const vector<HdfsFileDesc*>& files) {
-  vector<ScanRange*> footer_ranges;
-  for (int i = 0; i < files.size(); ++i) {
+  for (HdfsFileDesc* file : files) {
     // If the file size is less than 12 bytes, it is an invalid Parquet file.
-    if (files[i]->file_length < 12) {
+    if (file->file_length < 12) {
       return Status(Substitute("Parquet file $0 has an invalid file length: $1",
-          files[i]->filename, files[i]->file_length));
-    }
-    // Compute the offset of the file footer.
-    int64_t footer_size = min(FOOTER_SIZE, files[i]->file_length);
-    int64_t footer_start = files[i]->file_length - footer_size;
-    DCHECK_GE(footer_start, 0);
-
-    // Try to find the split with the footer.
-    ScanRange* footer_split = FindFooterSplit(files[i]);
-
-    for (int j = 0; j < files[i]->splits.size(); ++j) {
-      ScanRange* split = files[i]->splits[j];
-
-      DCHECK_LE(split->offset() + split->len(), files[i]->file_length);
-      // If there are no materialized slots (such as count(*) over the table), we can
-      // get the result with the file metadata alone and don't need to read any row
-      // groups. We only want a single node to process the file footer in this case,
-      // which is the node with the footer split.  If it's not a count(*), we create a
-      // footer range for the split always.
-      if (!scan_node->IsZeroSlotTableScan() || footer_split == split) {
-        ScanRangeMetadata* split_metadata =
-            static_cast<ScanRangeMetadata*>(split->meta_data());
-        // Each split is processed by first issuing a scan range for the file footer, which
-        // is done here, followed by scan ranges for the columns of each row group within
-        // the actual split (in InitColumns()). The original split is stored in the
-        // metadata associated with the footer range.
-        ScanRange* footer_range;
-        if (footer_split != nullptr) {
-          footer_range = scan_node->AllocateScanRange(files[i]->fs,
-              files[i]->filename.c_str(), footer_size, footer_start,
-              split_metadata->partition_id, footer_split->disk_id(),
-              footer_split->expected_local(),
-              BufferOpts(footer_split->try_cache(), files[i]->mtime), split);
-        } else {
-          // If we did not find the last split, we know it is going to be a remote read.
-          footer_range =
-              scan_node->AllocateScanRange(files[i]->fs, files[i]->filename.c_str(),
-                  footer_size, footer_start, split_metadata->partition_id, -1, false,
-                  BufferOpts::Uncached(), split);
-        }
-
-        footer_ranges.push_back(footer_range);
-      } else {
-        scan_node->RangeComplete(THdfsFileFormat::PARQUET, THdfsCompression::NONE);
-      }
+          file->filename, file->file_length));
     }
   }
-  // The threads that process the footer will also do the scan, so we mark all the files
-  // as complete here.
-  RETURN_IF_ERROR(scan_node->AddDiskIoRanges(footer_ranges, files.size()));
-  return Status::OK();
-}
-
-ScanRange* HdfsParquetScanner::FindFooterSplit(HdfsFileDesc* file) {
-  DCHECK(file != nullptr);
-  for (int i = 0; i < file->splits.size(); ++i) {
-    ScanRange* split = file->splits[i];
-    if (split->offset() + split->len() == file->file_length) return split;
-  }
-  return nullptr;
+  return IssueFooterRanges(scan_node, THdfsFileFormat::PARQUET, files);
 }
 
 namespace impala {
@@ -379,18 +312,6 @@ int HdfsParquetScanner::CountScalarColumns(const vector<ParquetColumnReader*>& c
   return num_columns;
 }
 
-void HdfsParquetScanner::CheckFiltersEffectiveness() {
-  for (int i = 0; i < filter_stats_.size(); ++i) {
-    LocalFilterStats* stats = &filter_stats_[i];
-    const RuntimeFilter* filter = filter_ctxs_[i]->filter;
-    double reject_ratio = stats->rejected / static_cast<double>(stats->considered);
-    if (filter->AlwaysTrue() ||
-        reject_ratio < FLAGS_parquet_min_filter_reject_ratio) {
-      stats->enabled = 0;
-    }
-  }
-}
-
 Status HdfsParquetScanner::ProcessSplit() {
   DCHECK(scan_node_->HasRowBatchQueue());
   HdfsScanNode* scan_node = static_cast<HdfsScanNode*>(scan_node_);
@@ -1126,104 +1047,6 @@ Status HdfsParquetScanner::Codegen(HdfsScanNodeBase* node,
   return Status::OK();
 }
 
-bool HdfsParquetScanner::EvalRuntimeFilters(TupleRow* row) {
-  int num_filters = filter_ctxs_.size();
-  for (int i = 0; i < num_filters; ++i) {
-    if (!EvalRuntimeFilter(i, row)) return false;
-  }
-  return true;
-}
-
-// ; Function Attrs: noinline
-// define i1 @EvalRuntimeFilters(%"class.impala::HdfsParquetScanner"* %this,
-//                               %"class.impala::TupleRow"* %row) #34 {
-// entry:
-//   %0 = call i1 @_ZN6impala18HdfsParquetScanner17EvalRuntimeFilterEiPNS_8TupleRowE.2(
-//       %"class.impala::HdfsParquetScanner"* %this, i32 0, %"class.impala::TupleRow"*
-//       %row)
-//   br i1 %0, label %continue, label %bail_out
-//
-// bail_out:                                         ; preds = %entry
-//   ret i1 false
-//
-// continue:                                         ; preds = %entry
-//   ret i1 true
-// }
-//
-// EvalRuntimeFilter() is the same as the cross-compiled version except EvalOneFilter()
-// is replaced with the one generated by CodegenEvalOneFilter().
-Status HdfsParquetScanner::CodegenEvalRuntimeFilters(
-    LlvmCodeGen* codegen, const vector<ScalarExpr*>& filter_exprs, llvm::Function** fn) {
-  llvm::LLVMContext& context = codegen->context();
-  LlvmBuilder builder(context);
-
-  *fn = nullptr;
-  llvm::Type* this_type = codegen->GetStructPtrType<HdfsParquetScanner>();
-  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
-  LlvmCodeGen::FnPrototype prototype(codegen, "EvalRuntimeFilters",
-      codegen->bool_type());
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("this", this_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
-
-  llvm::Value* args[2];
-  llvm::Function* eval_runtime_filters_fn = prototype.GeneratePrototype(&builder, args);
-  llvm::Value* this_arg = args[0];
-  llvm::Value* row_arg = args[1];
-
-  int num_filters = filter_exprs.size();
-  if (num_filters == 0) {
-    builder.CreateRet(codegen->true_value());
-  } else {
-    // row_rejected_block: jump target for when a filter is evaluated to false.
-    llvm::BasicBlock* row_rejected_block =
-        llvm::BasicBlock::Create(context, "row_rejected", eval_runtime_filters_fn);
-
-    DCHECK_GT(num_filters, 0);
-    for (int i = 0; i < num_filters; ++i) {
-      llvm::Function* eval_runtime_filter_fn =
-          codegen->GetFunction(IRFunction::PARQUET_SCANNER_EVAL_RUNTIME_FILTER, true);
-      DCHECK(eval_runtime_filter_fn != nullptr);
-
-      // Codegen function for inlining filter's expression evaluation and constant fold
-      // the type of the expression into the hashing function to avoid branches.
-      llvm::Function* eval_one_filter_fn;
-      DCHECK(filter_exprs[i] != nullptr);
-      RETURN_IF_ERROR(FilterContext::CodegenEval(codegen, filter_exprs[i],
-          &eval_one_filter_fn));
-      DCHECK(eval_one_filter_fn != nullptr);
-
-      int replaced = codegen->ReplaceCallSites(eval_runtime_filter_fn, eval_one_filter_fn,
-          "FilterContext4Eval");
-      DCHECK_EQ(replaced, 1);
-
-      llvm::Value* idx = codegen->GetI32Constant(i);
-      llvm::Value* passed_filter = builder.CreateCall(
-          eval_runtime_filter_fn, llvm::ArrayRef<llvm::Value*>({this_arg, idx, row_arg}));
-
-      llvm::BasicBlock* continue_block =
-          llvm::BasicBlock::Create(context, "continue", eval_runtime_filters_fn);
-      builder.CreateCondBr(passed_filter, continue_block, row_rejected_block);
-      builder.SetInsertPoint(continue_block);
-    }
-    builder.CreateRet(codegen->true_value());
-
-    builder.SetInsertPoint(row_rejected_block);
-    builder.CreateRet(codegen->false_value());
-
-    // Don't inline this function to avoid code bloat in ProcessScratchBatch().
-    // If there is any filter, EvalRuntimeFilters() is large enough to not benefit
-    // much from inlining.
-    eval_runtime_filters_fn->addFnAttr(llvm::Attribute::NoInline);
-  }
-
-  *fn = codegen->FinalizeFunction(eval_runtime_filters_fn);
-  if (*fn == nullptr) {
-    return Status("Codegen'd HdfsParquetScanner::EvalRuntimeFilters() failed "
-        "verification, see log");
-  }
-  return Status::OK();
-}
-
 bool HdfsParquetScanner::AssembleCollection(
     const vector<ParquetColumnReader*>& column_readers, int new_collection_rep_level,
     CollectionValueBuilder* coll_value_builder) {

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-parquet-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.h b/be/src/exec/hdfs-parquet-scanner.h
index ccb109c..7fede3b 100644
--- a/be/src/exec/hdfs-parquet-scanner.h
+++ b/be/src/exec/hdfs-parquet-scanner.h
@@ -20,7 +20,6 @@
 #define IMPALA_EXEC_HDFS_PARQUET_SCANNER_H
 
 #include "codegen/impala-ir.h"
-#include "common/global-flags.h"
 #include "exec/hdfs-scanner.h"
 #include "exec/parquet-common.h"
 #include "exec/parquet-scratch-tuple-batch.h"
@@ -362,14 +361,6 @@ class HdfsParquetScanner : public HdfsScanner {
   friend class ScalarColumnReader;
   friend class BoolColumnReader;
 
-  /// Size of the file footer.  This is a guess.  If this value is too little, we will
-  /// need to issue another read.
-  static const int64_t FOOTER_SIZE = 1024 * 100;
-  static_assert(FOOTER_SIZE <= READ_SIZE_MIN_VALUE,
-      "FOOTER_SIZE can not be greater than READ_SIZE_MIN_VALUE.\n"
-      "You can increase FOOTER_SIZE if you want, "
-      "just don't forget to increase READ_SIZE_MIN_VALUE as well.");
-
   /// Index of the current row group being processed. Initialized to -1 which indicates
   /// that we have not started processing the first row group yet (GetNext() has not yet
   /// been called).
@@ -391,41 +382,11 @@ class HdfsParquetScanner : public HdfsScanner {
   /// the scanner. Stored in 'obj_pool_'.
   vector<ScalarExprEvaluator*> min_max_conjunct_evals_;
 
-  /// Cached runtime filter contexts, one for each filter that applies to this column,
-  /// owned by instances of this class.
-  vector<const FilterContext*> filter_ctxs_;
-
-  struct LocalFilterStats {
-    /// Total number of rows to which each filter was applied
-    int64_t considered;
-
-    /// Total number of rows that each filter rejected.
-    int64_t rejected;
-
-    /// Total number of rows that each filter could have been applied to (if it were
-    /// available from row 0).
-    int64_t total_possible;
-
-    /// Use known-width type to act as logical boolean.  Set to 1 if corresponding filter
-    /// in filter_ctxs_ should be applied, 0 if it was ineffective and was disabled.
-    uint8_t enabled;
-
-    /// Padding to ensure structs do not straddle cache-line boundary.
-    uint8_t padding[7];
-
-    LocalFilterStats() : considered(0), rejected(0), total_possible(0), enabled(1) { }
-  };
-
   /// Pool used for allocating caches of definition/repetition levels and tuples for
   /// dictionary filtering. The definition/repetition levels are populated by the
   /// level readers. The pool is freed in Close().
   boost::scoped_ptr<MemPool> perm_pool_;
 
-  /// Track statistics of each filter (one for each filter in filter_ctxs_) per scanner so
-  /// that expensive aggregation up to the scan node can be performed once, during
-  /// Close().
-  vector<LocalFilterStats> filter_stats_;
-
   /// Number of scratch batches processed so far.
   int64_t row_batches_produced_;
 
@@ -511,10 +472,6 @@ class HdfsParquetScanner : public HdfsScanner {
   Status EvaluateStatsConjuncts(const parquet::FileMetaData& file_metadata,
       const parquet::RowGroup& row_group, bool* skip_row_group) WARN_UNUSED_RESULT;
 
-  /// Check runtime filters' effectiveness every BATCHES_PER_FILTER_SELECTIVITY_CHECK
-  /// row batches. Will update 'filter_stats_'.
-  void CheckFiltersEffectiveness();
-
   /// Advances 'row_group_idx_' to the next non-empty row group and initializes
   /// the column readers to scan it. Recoverable errors are logged to the runtime
   /// state. Only returns a non-OK status if a non-recoverable error is encountered
@@ -548,24 +505,6 @@ class HdfsParquetScanner : public HdfsScanner {
   /// materialized tuples. This is a separate function so it can be codegened.
   int ProcessScratchBatch(RowBatch* dst_batch);
 
-  /// Evaluates 'row' against the i-th runtime filter for this scan node and returns
-  /// true if 'row' finds a match in the filter. Returns false otherwise.
-  bool EvalRuntimeFilter(int i, TupleRow* row);
-
-  /// Evaluates runtime filters (if any) against the given row. Returns true if
-  /// they passed, false otherwise. Maintains the runtime filter stats, determines
-  /// whether the filters are effective, and disables them if they are not. This is
-  /// replaced by generated code at runtime.
-  bool EvalRuntimeFilters(TupleRow* row);
-
-  /// Codegen EvalRuntimeFilters() by unrolling the loop in the interpreted version
-  /// and emitting a customized version of EvalRuntimeFilter() for each filter in
-  /// 'filter_ctxs'. Return error status on failure. The generated function is returned
-  /// via 'fn'.
-  static Status CodegenEvalRuntimeFilters(LlvmCodeGen* codegen,
-      const std::vector<ScalarExpr*>& filter_exprs, llvm::Function** fn)
-      WARN_UNUSED_RESULT;
-
   /// Reads data using 'column_readers' to materialize the tuples of a CollectionValue
   /// allocated from 'coll_value_builder'. Increases 'coll_items_read_counter_' by the
   /// number of items in this collection and descendant collections.
@@ -592,10 +531,6 @@ class HdfsParquetScanner : public HdfsScanner {
   inline bool ReadCollectionItem(const std::vector<ParquetColumnReader*>& column_readers,
       bool materialize_tuple, MemPool* pool, Tuple* tuple) const;
 
-  /// Find and return the last split in the file if it is assigned to this scan node.
-  /// Returns NULL otherwise.
-  static io::ScanRange* FindFooterSplit(HdfsFileDesc* file);
-
   /// Process the file footer and parse file_metadata_.  This should be called with the
   /// last FOOTER_SIZE bytes in context_.
   Status ProcessFooter() WARN_UNUSED_RESULT;

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 98c6e14..21a6357 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -23,6 +23,7 @@
 #include "exec/hdfs-rcfile-scanner.h"
 #include "exec/hdfs-avro-scanner.h"
 #include "exec/hdfs-parquet-scanner.h"
+#include "exec/hdfs-orc-scanner.h"
 
 #include <avro/errors.h>
 #include <avro/schema.h>
@@ -451,6 +452,8 @@ Status HdfsScanNodeBase::IssueInitialScanRanges(RuntimeState* state) {
   // Issue initial ranges for all file types.
   RETURN_IF_ERROR(HdfsParquetScanner::IssueInitialRanges(this,
       matching_per_type_files[THdfsFileFormat::PARQUET]));
+  RETURN_IF_ERROR(HdfsOrcScanner::IssueInitialRanges(this,
+      matching_per_type_files[THdfsFileFormat::ORC]));
   RETURN_IF_ERROR(HdfsTextScanner::IssueInitialRanges(this,
       matching_per_type_files[THdfsFileFormat::TEXT]));
   RETURN_IF_ERROR(BaseSequenceScanner::IssueInitialRanges(this,
@@ -582,6 +585,9 @@ Status HdfsScanNodeBase::CreateAndOpenScanner(HdfsPartitionDescriptor* partition
     case THdfsFileFormat::PARQUET:
       scanner->reset(new HdfsParquetScanner(this, runtime_state_));
       break;
+    case THdfsFileFormat::ORC:
+      scanner->reset(new HdfsOrcScanner(this, runtime_state_));
+      break;
     default:
       return Status(Substitute("Unknown Hdfs file format type: $0",
           partition->file_format()));

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-scan-node-mt.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-mt.cc b/be/src/exec/hdfs-scan-node-mt.cc
index 7ea4d80..d143e91 100644
--- a/be/src/exec/hdfs-scan-node-mt.cc
+++ b/be/src/exec/hdfs-scan-node-mt.cc
@@ -46,6 +46,7 @@ Status HdfsScanNodeMt::Prepare(RuntimeState* state) {
   // because the scanner of the corresponding file format does implement GetNext().
   for (const auto& files: per_type_files_) {
     if (!files.second.empty() && files.first != THdfsFileFormat::PARQUET
+        && files.first != THdfsFileFormat::ORC
         && files.first != THdfsFileFormat::TEXT) {
       stringstream msg;
       msg << "Unsupported file format with HdfsScanNodeMt: " << files.first;

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-scanner-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner-ir.cc b/be/src/exec/hdfs-scanner-ir.cc
index ec1d2a3..0d34356 100644
--- a/be/src/exec/hdfs-scanner-ir.cc
+++ b/be/src/exec/hdfs-scanner-ir.cc
@@ -95,6 +95,20 @@ void StringToDecimalSymbolDummy() {
   StringToDecimal16(nullptr, 0, 0, 0, false, nullptr);
 }
 
+bool HdfsScanner::EvalRuntimeFilter(int i, TupleRow* row) {
+  LocalFilterStats* stats = &filter_stats_[i];
+  const FilterContext* ctx = filter_ctxs_[i];
+  ++stats->total_possible;
+  if (stats->enabled && ctx->filter->HasFilter()) {
+    ++stats->considered;
+    if (!ctx->Eval(row)) {
+      ++stats->rejected;
+      return false;
+    }
+  }
+  return true;
+}
+
 // Define the string parsing functions for llvm.  Stamp out the templated functions
 #ifdef IR_COMPILE
 using ParseResult = StringParser::ParseResult;

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index 1809fe5..a4aee4d 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -35,10 +35,15 @@
 #include "common/names.h"
 
 using namespace impala;
+using namespace impala::io;
 using namespace strings;
 
+DEFINE_double(min_filter_reject_ratio, 0.1, "(Advanced) If the percentage of "
+    "rows rejected by a runtime filter drops below this value, the filter is disabled.");
+
 const char* FieldLocation::LLVM_CLASS_NAME = "struct.impala::FieldLocation";
 const char* HdfsScanner::LLVM_CLASS_NAME = "class.impala::HdfsScanner";
+const int64_t HdfsScanner::FOOTER_SIZE;
 
 HdfsScanner::HdfsScanner(HdfsScanNodeBase* scan_node, RuntimeState* state)
     : scan_node_(scan_node),
@@ -587,6 +592,96 @@ Status HdfsScanner::CodegenInitTuple(
   return Status::OK();
 }
 
+// ; Function Attrs: noinline
+// define i1 @EvalRuntimeFilters(%"class.impala::HdfsScanner"* %this,
+//                               %"class.impala::TupleRow"* %row) #34 {
+// entry:
+//   %0 = call i1 @_ZN6impala11HdfsScanner17EvalRuntimeFilterEiPNS_8TupleRowE.2(
+//       %"class.impala::HdfsScanner"* %this, i32 0, %"class.impala::TupleRow"*
+//       %row)
+//   br i1 %0, label %continue, label %bail_out
+//
+// bail_out:                                         ; preds = %entry
+//   ret i1 false
+//
+// continue:                                         ; preds = %entry
+//   ret i1 true
+// }
+//
+// EvalRuntimeFilter() is the same as the cross-compiled version except EvalOneFilter()
+// is replaced with the one generated by CodegenEvalOneFilter().
+Status HdfsScanner::CodegenEvalRuntimeFilters(
+    LlvmCodeGen* codegen, const vector<ScalarExpr*>& filter_exprs, llvm::Function** fn) {
+  llvm::LLVMContext& context = codegen->context();
+  LlvmBuilder builder(context);
+
+  *fn = nullptr;
+  llvm::Type* this_type = codegen->GetStructPtrType<HdfsScanner>();
+  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
+  LlvmCodeGen::FnPrototype prototype(codegen, "EvalRuntimeFilters",
+                                     codegen->bool_type());
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("this", this_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
+
+  llvm::Value* args[2];
+  llvm::Function* eval_runtime_filters_fn = prototype.GeneratePrototype(&builder, args);
+  llvm::Value* this_arg = args[0];
+  llvm::Value* row_arg = args[1];
+
+  int num_filters = filter_exprs.size();
+  if (num_filters == 0) {
+    builder.CreateRet(codegen->true_value());
+  } else {
+    // row_rejected_block: jump target for when a filter is evaluated to false.
+    llvm::BasicBlock* row_rejected_block =
+        llvm::BasicBlock::Create(context, "row_rejected", eval_runtime_filters_fn);
+
+    DCHECK_GT(num_filters, 0);
+    for (int i = 0; i < num_filters; ++i) {
+      llvm::Function* eval_runtime_filter_fn =
+          codegen->GetFunction(IRFunction::HDFS_SCANNER_EVAL_RUNTIME_FILTER, true);
+      DCHECK(eval_runtime_filter_fn != nullptr);
+
+      // Codegen function for inlining filter's expression evaluation and constant fold
+      // the type of the expression into the hashing function to avoid branches.
+      llvm::Function* eval_one_filter_fn;
+      DCHECK(filter_exprs[i] != nullptr);
+      RETURN_IF_ERROR(FilterContext::CodegenEval(codegen, filter_exprs[i],
+          &eval_one_filter_fn));
+      DCHECK(eval_one_filter_fn != nullptr);
+
+      int replaced = codegen->ReplaceCallSites(eval_runtime_filter_fn, eval_one_filter_fn,
+          "FilterContext4Eval");
+      DCHECK_EQ(replaced, 1);
+
+      llvm::Value* idx = codegen->GetI32Constant(i);
+      llvm::Value* passed_filter = builder.CreateCall(
+          eval_runtime_filter_fn, llvm::ArrayRef<llvm::Value*>({this_arg, idx, row_arg}));
+
+      llvm::BasicBlock* continue_block =
+          llvm::BasicBlock::Create(context, "continue", eval_runtime_filters_fn);
+      builder.CreateCondBr(passed_filter, continue_block, row_rejected_block);
+      builder.SetInsertPoint(continue_block);
+    }
+    builder.CreateRet(codegen->true_value());
+
+    builder.SetInsertPoint(row_rejected_block);
+    builder.CreateRet(codegen->false_value());
+
+    // Don't inline this function to avoid code bloat in ProcessScratchBatch().
+    // If there is any filter, EvalRuntimeFilters() is large enough to not benefit
+    // much from inlining.
+    eval_runtime_filters_fn->addFnAttr(llvm::Attribute::NoInline);
+  }
+
+  *fn = codegen->FinalizeFunction(eval_runtime_filters_fn);
+  if (*fn == nullptr) {
+    return Status("Codegen'd HdfsScanner::EvalRuntimeFilters() failed "
+        "verification, see log");
+  }
+  return Status::OK();
+}
+
 Status HdfsScanner::UpdateDecompressor(const THdfsCompression::type& compression) {
   // Check whether the file in the stream has different compression from the last one.
   if (compression != decompression_type_) {
@@ -671,3 +766,87 @@ void HdfsScanner::ReportColumnParseError(const SlotDescriptor* desc,
     if (state_->abort_on_error() && parse_status_.ok()) parse_status_ = Status(ss.str());
   }
 }
+
+void HdfsScanner::CheckFiltersEffectiveness() {
+  for (int i = 0; i < filter_stats_.size(); ++i) {
+    LocalFilterStats* stats = &filter_stats_[i];
+    const RuntimeFilter* filter = filter_ctxs_[i]->filter;
+    double reject_ratio = stats->rejected / static_cast<double>(stats->considered);
+    if (filter->AlwaysTrue() ||
+        reject_ratio < FLAGS_min_filter_reject_ratio) {
+      stats->enabled = 0;
+    }
+  }
+}
+
+Status HdfsScanner::IssueFooterRanges(HdfsScanNodeBase* scan_node,
+    const THdfsFileFormat::type& file_type, const vector<HdfsFileDesc*>& files) {
+  vector<ScanRange*> footer_ranges;
+  for (int i = 0; i < files.size(); ++i) {
+    // Compute the offset of the file footer.
+    int64_t footer_size = min(FOOTER_SIZE, files[i]->file_length);
+    int64_t footer_start = files[i]->file_length - footer_size;
+    DCHECK_GE(footer_start, 0);
+
+    // Try to find the split with the footer.
+    ScanRange* footer_split = FindFooterSplit(files[i]);
+
+    for (int j = 0; j < files[i]->splits.size(); ++j) {
+      ScanRange* split = files[i]->splits[j];
+
+      DCHECK_LE(split->offset() + split->len(), files[i]->file_length);
+      // If there are no materialized slots (such as count(*) over the table), we can
+      // get the result with the file metadata alone and don't need to read any row
+      // groups. We only want a single node to process the file footer in this case,
+      // which is the node with the footer split.  If it's not a count(*), we create a
+      // footer range for the split always.
+      if (!scan_node->IsZeroSlotTableScan() || footer_split == split) {
+        ScanRangeMetadata* split_metadata =
+            static_cast<ScanRangeMetadata*>(split->meta_data());
+        // Each split is processed by first issuing a scan range for the file footer, which
+        // is done here, followed by scan ranges for the columns of each row group within
+        // the actual split (in InitColumns()). The original split is stored in the
+        // metadata associated with the footer range.
+        ScanRange* footer_range;
+        if (footer_split != nullptr) {
+          footer_range = scan_node->AllocateScanRange(files[i]->fs,
+              files[i]->filename.c_str(), footer_size, footer_start,
+              split_metadata->partition_id, footer_split->disk_id(),
+              footer_split->expected_local(),
+              BufferOpts(footer_split->try_cache(), files[i]->mtime), split);
+        } else {
+          // If we did not find the last split, we know it is going to be a remote read.
+          footer_range =
+              scan_node->AllocateScanRange(files[i]->fs, files[i]->filename.c_str(),
+                   footer_size, footer_start, split_metadata->partition_id, -1, false,
+                   BufferOpts::Uncached(), split);
+        }
+
+        footer_ranges.push_back(footer_range);
+      } else {
+        scan_node->RangeComplete(file_type, THdfsCompression::NONE);
+      }
+    }
+  }
+  // The threads that process the footer will also do the scan, so we mark all the files
+  // as complete here.
+  RETURN_IF_ERROR(scan_node->AddDiskIoRanges(footer_ranges, files.size()));
+  return Status::OK();
+}
+
+ScanRange* HdfsScanner::FindFooterSplit(HdfsFileDesc* file) {
+  DCHECK(file != nullptr);
+  for (int i = 0; i < file->splits.size(); ++i) {
+    ScanRange* split = file->splits[i];
+    if (split->offset() + split->len() == file->file_length) return split;
+  }
+  return nullptr;
+}
+
+bool HdfsScanner::EvalRuntimeFilters(TupleRow* row) {
+  int num_filters = filter_ctxs_.size();
+  for (int i = 0; i < num_filters; ++i) {
+    if (!EvalRuntimeFilter(i, row)) return false;
+  }
+  return true;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/exec/hdfs-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.h b/be/src/exec/hdfs-scanner.h
index 6497457..73b8b70 100644
--- a/be/src/exec/hdfs-scanner.h
+++ b/be/src/exec/hdfs-scanner.h
@@ -25,6 +25,7 @@
 #include <boost/scoped_ptr.hpp>
 
 #include "codegen/impala-ir.h"
+#include "common/global-flags.h"
 #include "common/object-pool.h"
 #include "common/status.h"
 #include "exec/hdfs-scan-node-base.h"
@@ -44,6 +45,12 @@ class TextConverter;
 class TupleDescriptor;
 class SlotDescriptor;
 
+// The number of row batches between checks to see if a filter is effective, and
+// should be disabled. Must be a power of two.
+constexpr int BATCHES_PER_FILTER_SELECTIVITY_CHECK = 16;
+static_assert(BitUtil::IsPowerOf2(BATCHES_PER_FILTER_SELECTIVITY_CHECK),
+              "BATCHES_PER_FILTER_SELECTIVITY_CHECK must be a power of two");
+
 /// Intermediate structure used for two pass parsing approach. In the first pass,
 /// the FieldLocation structs are filled out and contain where all the fields start and
 /// their lengths.  In the second pass, the FieldLocation is used to write out the
@@ -287,6 +294,67 @@ class HdfsScanner {
   /// Jitted write tuples function pointer.  Null if codegen is disabled.
   WriteTuplesFn write_tuples_fn_ = nullptr;
 
+  struct LocalFilterStats {
+    /// Total number of rows to which each filter was applied
+    int64_t considered;
+
+    /// Total number of rows that each filter rejected.
+    int64_t rejected;
+
+    /// Total number of rows that each filter could have been applied to (if it were
+    /// available from row 0).
+    int64_t total_possible;
+
+    /// Use known-width type to act as logical boolean.  Set to 1 if corresponding filter
+    /// in filter_ctxs_ should be applied, 0 if it was ineffective and was disabled.
+    uint8_t enabled;
+
+    /// Padding to ensure structs do not straddle cache-line boundary.
+    uint8_t padding[7];
+
+    LocalFilterStats() : considered(0), rejected(0), total_possible(0), enabled(1) { }
+  };
+
+  /// Cached runtime filter contexts, one for each filter that applies to this column.
+  vector<const FilterContext *> filter_ctxs_;
+
+  /// Track statistics of each filter (one for each filter in filter_ctxs_) per scanner
+  /// so that expensive aggregation up to the scan node can be performed once, during
+  /// Close().
+  vector<LocalFilterStats> filter_stats_;
+
+  /// Size of the file footer for ORC and Parquet. This is a guess. If this value is too
+  /// little, we will need to issue another read.
+  static const int64_t FOOTER_SIZE = 1024 * 100;
+  static_assert(FOOTER_SIZE <= READ_SIZE_MIN_VALUE,
+      "FOOTER_SIZE can not be greater than READ_SIZE_MIN_VALUE.\n"
+      "You can increase FOOTER_SIZE if you want, "
+      "just don't forget to increase READ_SIZE_MIN_VALUE as well.");
+
+  /// Check runtime filters' effectiveness every BATCHES_PER_FILTER_SELECTIVITY_CHECK
+  /// row batches. Will update 'filter_stats_'.
+  void CheckFiltersEffectiveness();
+
+  /// Evaluates 'row' against the i-th runtime filter for this scan node and returns
+  /// true if 'row' finds a match in the filter. Returns false otherwise.
+  bool EvalRuntimeFilter(int i, TupleRow* row);
+
+  /// Evaluates runtime filters (if any) against the given row. Returns true if
+  /// they passed, false otherwise. Maintains the runtime filter stats, determines
+  /// whether the filters are effective, and disables them if they are not. This is
+  /// replaced by generated code at runtime.
+  bool EvalRuntimeFilters(TupleRow* row);
+
+  /// Find and return the last split in the file if it is assigned to this scan node.
+  /// Returns NULL otherwise.
+  static io::ScanRange* FindFooterSplit(HdfsFileDesc* file);
+
+  /// Issue just the footer range for each file. This function is only used in parquet
+  /// and orc scanners. We'll then parse the footer and pick out the columns we want.
+  static Status IssueFooterRanges(HdfsScanNodeBase* scan_node,
+      const THdfsFileFormat::type& file_type, const std::vector<HdfsFileDesc*>& files)
+      WARN_UNUSED_RESULT;
+
   /// Implements GetNext(). Should be overridden by subclasses.
   /// Only valid to call if the parent scan node is multi-threaded.
   virtual Status GetNextInternal(RowBatch* row_batch) WARN_UNUSED_RESULT {
@@ -420,6 +488,14 @@ class HdfsScanner {
   static Status CodegenInitTuple(
       const HdfsScanNodeBase* node, LlvmCodeGen* codegen, llvm::Function** init_tuple_fn);
 
+  /// Codegen EvalRuntimeFilters() by unrolling the loop in the interpreted version
+  /// and emitting a customized version of EvalRuntimeFilter() for each filter in
+  /// 'filter_ctxs'. Return error status on failure. The generated function is returned
+  /// via 'fn'.
+  static Status CodegenEvalRuntimeFilters(LlvmCodeGen* codegen,
+      const std::vector<ScalarExpr*>& filter_exprs, llvm::Function** fn)
+      WARN_UNUSED_RESULT;
+
   /// Report parse error for column @ desc.   If abort_on_error is true, sets
   /// parse_status_ to the error message.
   void ReportColumnParseError(const SlotDescriptor* desc, const char* data, int len);

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/be/src/util/backend-gflag-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index 0bbaa89..02e1ed8 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -27,6 +27,7 @@
 DECLARE_bool(load_catalog_in_background);
 DECLARE_bool(load_auth_to_local_rules);
 DECLARE_bool(enable_stats_extrapolation);
+DECLARE_bool(enable_orc_scanner);
 DECLARE_int32(non_impala_java_vlog);
 DECLARE_int32(num_metadata_loading_threads);
 DECLARE_int32(max_hdfs_partitions_parallel_load);
@@ -56,6 +57,7 @@ Status GetThriftBackendGflags(JNIEnv* jni_env, jbyteArray* cfg_bytes) {
   TBackendGflags cfg;
   cfg.__set_authorization_policy_file(FLAGS_authorization_policy_file);
   cfg.__set_load_catalog_in_background(FLAGS_load_catalog_in_background);
+  cfg.__set_enable_orc_scanner(FLAGS_enable_orc_scanner);
   cfg.__set_server_name(FLAGS_server_name);
   cfg.__set_sentry_config(FLAGS_sentry_config);
   cfg.__set_authorization_policy_provider_class(

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/bin/bootstrap_toolchain.py
----------------------------------------------------------------------
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index a09c905..f54bf04 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -429,7 +429,7 @@ if __name__ == "__main__":
   packages = map(Package, ["llvm", "kudu",
       "avro", "binutils", "boost", "breakpad", "bzip2", "cmake", "crcutil",
       "flatbuffers", "gcc", "gflags", "glog", "gperftools", "gtest", "libev",
-      "lz4", "openldap", "openssl", "protobuf",
+      "lz4", "openldap", "openssl", "orc", "protobuf",
       "rapidjson", "re2", "snappy", "thrift", "tpc-h", "tpc-ds", "zlib"])
   packages.insert(0, Package("llvm", "5.0.1-asserts"))
   bootstrap(toolchain_root, packages)

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 0ae7713..343610f 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -127,6 +127,8 @@ export IMPALA_OPENLDAP_VERSION=2.4.25
 unset IMPALA_OPENLDAP_URL
 export IMPALA_OPENSSL_VERSION=1.0.2l
 unset IMPALA_OPENSSL_URL
+export IMPALA_ORC_VERSION=1.4.3-p2
+unset IMPALA_ORC_URL
 export IMPALA_PROTOBUF_VERSION=2.6.1
 unset IMPALA_PROTOBUF_URL
 export IMPALA_POSTGRES_JDBC_DRIVER_VERSION=9.0-801

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/cmake_modules/FindOrc.cmake
----------------------------------------------------------------------
diff --git a/cmake_modules/FindOrc.cmake b/cmake_modules/FindOrc.cmake
new file mode 100644
index 0000000..ef06396
--- /dev/null
+++ b/cmake_modules/FindOrc.cmake
@@ -0,0 +1,55 @@
+##############################################################################
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+##############################################################################
+
+# - Find Orc (headers and liborc.a) with ORC_ROOT hinting a location
+# This module defines
+#  ORC_INCLUDE_DIR, directory containing headers
+#  ORC_STATIC_LIB, path to liborc.a
+#  ORC_FOUND
+set(ORC_SEARCH_HEADER_PATHS
+  ${ORC_ROOT}/include
+  $ENV{IMPALA_HOME}/thirdparty/orc-$ENV{IMPALA_ORC_VERSION}/build/include)
+
+set(ORC_SEARCH_LIB_PATH
+  ${ORC_ROOT}/lib
+  $ENV{IMPALA_HOME}/thirdparty/orc-$ENV{IMPALA_ORC_VERSION}/build/lib)
+
+find_path(ORC_INCLUDE_DIR NAMES orc/OrcFile.hh OrcFile.hh PATHS
+  ${ORC_SEARCH_HEADER_PATHS}
+  # make sure we don't accidentally pick up a different version
+  NO_DEFAULT_PATH)
+
+find_library(ORC_STATIC_LIB NAMES liborc.a PATHS ${ORC_SEARCH_LIB_PATH})
+
+if(NOT ORC_STATIC_LIB)
+  message(FATAL_ERROR "ORC includes and libraries NOT found. "
+    "Looked for headers in ${ORC_SEARCH_HEADER_PATHS}, "
+    "and for libs in ${ORC_SEARCH_LIB_PATH}")
+  set(ORC_FOUND FALSE)
+else()
+  set(ORC_FOUND TRUE)
+endif ()
+
+set(ORC_FOUND ${ORC_STATIC_LIB_FOUND})
+
+mark_as_advanced(
+  ORC_INCLUDE_DIR
+  ORC_STATIC_LIB
+  ORC_FOUND
+)

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/common/thrift/BackendGflags.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index 412ca06..c98f50a 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -73,4 +73,6 @@ struct TBackendGflags {
   23: required double max_filter_error_rate
 
   24: required i64 min_buffer_size
+
+  25: required bool enable_orc_scanner
 }


[2/6] impala git commit: IMPALA-5903: Inconsistent specification of result set and result set metadata

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
index 204fc68..394157c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
@@ -84,6 +84,7 @@ create table tab (a int not null primary key)
 partition by range (a) (partition value = false)
 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Invalid hostname
@@ -107,6 +108,7 @@ Couldn't resolve this master's address bogus.host.name:7051
 create table tdata_master_addresses_whitespace (id int primary key) stored as kudu
   tblproperties('kudu.master_addresses' = '  localhost  ')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tdata_master_addresses_whitespace values (0), (1)
@@ -120,6 +122,7 @@ create table ignore_column_case (Id int, NAME string, vAlf float, vali bigint,
   primary key (Id, NAME)) PARTITION BY RANGE (PARTITION VALUE = (1, 'Martin'))
   STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into ignore_column_case values (1, 'Martin', 1.0, 10);
@@ -145,6 +148,7 @@ create table tbl_with_null_defaults (x int primary key, i1 tinyint default null,
   valdec8 decimal(18) default null, valdec16 decimal(38) default null)
   partition by hash (x) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tbl_with_null_defaults (x) values (1);
@@ -181,6 +185,7 @@ partition by range (
   partition '2009-01-03 00:00:00' <= VALUES
 ) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -236,6 +241,7 @@ create table ts_default (i int primary key, ts1 timestamp,
                          ts2 timestamp default cast('2009-01-01 00:00:00' as timestamp))
 partition by hash(i) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into ts_default (i) values (1);
@@ -267,6 +273,7 @@ INT,TIMESTAMP,TIMESTAMP
 create table unpartitioned_kudu_table (col0 bigint primary key, col1 string)
 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ---- ERRORS
 Unpartitioned Kudu tables are inefficient for large data sizes.
 ====
@@ -312,6 +319,7 @@ create table create_decimal
  primary key (decimal_4))
 stored as kudu;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Create as select table with decimal columns and primary key
@@ -327,4 +335,4 @@ select * from ctas_decimal;
 132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889,1
 ---- TYPES
 DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
index 3abcdab..9fd7f32 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 100, PARTITION 100 <= VALUES < 1000,
   PARTITION 1000 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into table tdata values
@@ -385,6 +386,7 @@ STRING,BIGINT,TINYINT,SMALLINT,BOOLEAN,INT,DOUBLE,FLOAT
 create table impala_3454 (key_1 tinyint, key_2 bigint, PRIMARY KEY (key_1, key_2))
   PARTITION BY HASH PARTITIONS 3 STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into impala_3454 values

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
index 6bba77a..da1bfac 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # VALUES, single row, all target cols, no errors
@@ -295,6 +296,7 @@ create table allkeytypes (i1 tinyint, i2 smallint, i3 int, i4 bigint, name strin
   partition value = (2,2,2,2,'2','2009-01-01 00:02:00.100000000'),
   partition value = (3,3,3,3,'3','2009-01-01 00:03:00.300000000')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allkeytypes select cast(id as tinyint), smallint_col, int_col,
@@ -325,6 +327,7 @@ create table tbl_with_defaults (a int primary key, b int null default 10,
   i decimal(9, 2) default 1111.11) partition by hash (a)
   partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tbl_with_defaults (a, f) values (1, 1), (2, 2), (3, 3), (4, 4)
@@ -360,6 +363,7 @@ INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL
 ---- QUERY
 alter table tbl_with_defaults add columns (j int null, k int not null default 10000)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 select * from tbl_with_defaults
@@ -421,6 +425,7 @@ INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL,INT,INT
 create table multiple_partition_cols (x bigint, y bigint, z string, primary key(x, y))
 partition by hash(x, y) partitions 8 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # SELECT with constant

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
index 96cbc14..17d365b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
@@ -5,6 +5,7 @@ create table simple_hash (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash(id) partitions 4,
   hash(name) partitions 2 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash
@@ -29,6 +30,7 @@ create table range_part_bounds (id int, name string, valf float, vali bigint,
   (partition values <= 10, partition 10 < values <= 20, partition 20 < values)
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_bounds
@@ -48,6 +50,7 @@ create table range_part_single (id int, name string, valf float, vali bigint,
   (partition value = 1, partition value = 10, partition value = 100)
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_single
@@ -68,6 +71,7 @@ create table range_part_multiple_bounds (id int, name string, valf float,
   (partition values <= 10, partition 10 < values <= 20, partition 20 < values <= 30,
    partition value = 40, partition value = 50) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_multiple_bounds
@@ -89,6 +93,7 @@ create table range_part_multiple_cols (id int, name string, valf float, vali big
   (partition value = (10, 'martin'), partition value = (20, 'dimitris'),
    partition value = (30, 'matthew')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 -- Test printing of multiple column range partitioning
@@ -116,6 +121,7 @@ create table range_part_single_string_col (id int, name string, valf float,
   (partition values <= 'aaa', partition 'aaa' < values <= 'bbb',
    partition 'bbb' < values <= 'ccc', partition value = 'ddd') stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_single_string_col
@@ -135,6 +141,7 @@ create table simple_hash_range (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash(id) partitions 4, range(id, name)
   (partition value = (10, 'martin'), partition value = (20, 'alex')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash_range
@@ -182,6 +189,7 @@ INT,STRING,STRING,STRING,INT
 create table simple_hash_all_columns (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash partitions 4 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash_all_columns
@@ -202,6 +210,7 @@ create table simple_range_all_columns (id int, name string, valf float, vali big
   (partition value = (1, 'a'), partition value = (2, 'b'))
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_range_all_columns
@@ -220,6 +229,7 @@ create table range_complex_const_boundary_vals (x int, y int, primary key (x))
   partition factorial(4) < values < factorial(5), partition value = factorial(6))
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_complex_const_boundary_vals

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
index 697a3a3..23a43b5 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
@@ -4,6 +4,7 @@ create table simple (id int primary key, name string, valf float, vali bigint)
   partition by range (partition values < 10, partition 10 <= values < 30,
   partition 30 <= values) stored as kudu tblproperties('kudu.num_tablet_replicas' = '1')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
index 8520677..8b7759b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tdata values

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
index 1c12f33..5cdfef2 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
@@ -7,6 +7,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into table tdata values

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test b/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
index 64fdced..71ab444 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
@@ -6,6 +6,7 @@ location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' symbol='NoArgs';
 create function no_args2() returns string
 location '$FILESYSTEM_PREFIX/test-warehouse/udf_test/libTestUdfs.so' symbol='NoArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select no_args();

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/load.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/load.test b/testdata/workloads/functional-query/queries/QueryTest/load.test
index 060868f..4528380 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/load.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/load.test
@@ -3,11 +3,13 @@
 alter table functional.test_load add partition
 (year=2009, month=1)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 alter table functional.test_load add partition
 (year=2010, month=1)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Insert some data into one of the partitions, used to verify we are not clobbering

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test b/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
index 50e2666..1239eea 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
@@ -5,6 +5,7 @@ create external table tbl
 row format delimited fields terminated by ','
 location 'file://$IMPALA_HOME/testdata/data/local_tbl'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe tbl

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test b/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
index f90dcff..ea4fb0d 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
@@ -157,7 +157,7 @@ STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 # Tests no matching partition.
 alter table p1 partition (j=100) set location '$FILESYSTEM_PREFIX/test-warehouse/newtable';
 ---- RESULTS
-# TODO: IMPALA-6775
+'New location has been set.'
 ==== QUERY
 # Check nothing was updated.
 show partitions p1
@@ -204,4 +204,4 @@ alter table p1 partition (j=100) set row format delimited fields terminated by '
 'Updated 0 partition(s).'
 ---- TYPES
 STRING
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test b/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
index 926c5bf..4605458 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
@@ -29,6 +29,7 @@ BIGINT, STRING, INT, INT
 ---- QUERY
 alter table jointbl_test add columns(new_col string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -58,6 +59,7 @@ BIGINT, STRING, INT, INT, STRING
 ---- QUERY
 alter table jointbl_test add columns(new_int_col int)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -87,6 +89,7 @@ BIGINT, STRING, INT, INT, STRING, INT
 ---- QUERY
 alter table jointbl_test drop column new_int_col
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -116,10 +119,12 @@ BIGINT, STRING, INT, INT, STRING
 ---- QUERY
 alter table jointbl_test drop column new_col
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 alter table jointbl_test drop column alltypes_id
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 select * from jointbl_test

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test b/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
index 6bfa21c..a8d2a80 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
@@ -166,6 +166,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # TRUNCATE IF EXISTS does not fail on non existent table
 truncate table if exists non_existent;
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 # Create an unpartitioned table.
@@ -182,6 +183,7 @@ BIGINT
 # TRUNCATE IF EXISTS base scenario
 truncate table if exists t3;
 ---- RESULTS
+'Table has been truncated.'
 ====
 ---- QUERY
 # Verify that truncate was successful

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
index fb91070..252d85c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
@@ -4,6 +4,7 @@ create function if not exists hive_pi() returns double
 location '$FILESYSTEM_PREFIX/test-warehouse/hive-exec.jar'
 symbol='org.apache.hadoop.hive.ql.udf.UDFPI';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function if not exists foo() returns double
@@ -33,6 +34,7 @@ create function if not exists twenty_args(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='TwentyArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 # Verifies that interpretation can support up to 20 arguments
@@ -49,6 +51,7 @@ create function if not exists twenty_one_args(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='TwentyOneArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 # Verifies that interpretation fails with more than 20 arguments.
@@ -63,6 +66,7 @@ create function if not exists nine_args_ir(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/test-udfs.ll'
 symbol='NineArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select nine_args_ir(1,2,3,4,5,6,7,8,9);
@@ -74,12 +78,14 @@ create function if not exists bad_expr(double) returns boolean
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='BadExpr' prepare_fn='BadExprPrepare' close_fn='BadExprClose';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function if not exists bad_expr2(double) returns boolean
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='BadExpr' prepare_fn='BadExpr2Prepare' close_fn='BadExprClose';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select count(t1.int_col) from functional.alltypes t1 join functional.alltypes t2
@@ -120,4 +126,5 @@ drop function nine_args_ir(int, int, int, int, int, int, int, int, int);
 drop function bad_expr(double);
 drop function bad_expr2(double);
 ---- RESULTS
+'Function has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
index 98f116f..0461222 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
@@ -4,12 +4,14 @@
 create view $DATABASE.simple_view as
 select * from functional.alltypes
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that 'if not exists' swallows the error (view already exists)
 create view if not exists $DATABASE.simple_view as
 select * from functional.alltypesagg
 ---- RESULTS
+'View already exists.'
 ====
 ---- QUERY
 # Create another simple view with 'if not exists' on a subset of
@@ -18,12 +20,14 @@ create view if not exists
 $DATABASE.simple_view_sub (x, y comment 'hello', z) as
 select int_col, string_col, timestamp_col from functional.alltypes
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a view on a parquet table (Hive cannot create/read/write parquet)
 create view $DATABASE.parquet_view as
 select * from functional_parquet.alltypes where id < 20
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a complex view with predicates, joins, aggregates and order by
@@ -34,12 +38,14 @@ on a.id = b.id where a.bigint_col < 50
 group by b.string_col having count(a.bigint_col) > 1
 order by b.string_col limit 100
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a view on a view
 create view $DATABASE.view_view (aaa, bbb) as
 select * from $DATABASE.complex_view
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that the views are displayed by 'show tables'
@@ -155,6 +161,7 @@ bigint
 # Test dropping a view
 drop view $DATABASE.simple_view_sub
 ---- RESULTS
+'View has been dropped.'
 ====
 ---- QUERY
 # Test that the view is gone
@@ -169,15 +176,18 @@ show tables in $DATABASE
 # Test 'if exists' for dropping a view (view does not exist)
 drop view if exists $DATABASE.bad_view
 ---- RESULTS
+'View does not exist.'
 ====
 ---- QUERY
 # Test 'if exists' does not drop a table with same name
 create table $DATABASE.drop_tbl_test(a int)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 drop view if exists $DATABASE.drop_tbl_test
 ---- RESULTS
+'Drop view is not allowed on a table.'
 ====
 ---- QUERY
 # Test drop table 'if exists' does not drop a view with same name.
@@ -185,6 +195,7 @@ drop view if exists $DATABASE.drop_tbl_test
 # still be listed in the subsequent show tables output (as a view).
 drop table if exists $DATABASE.complex_view
 ---- RESULTS
+'Drop table is not allowed on a view.'
 ====
 ---- QUERY
 # Test that the table is present
@@ -200,11 +211,13 @@ show tables in $DATABASE
 # Test renaming a view
 alter view $DATABASE.view_view rename to $DATABASE.view_on_view
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Test renaming a parquet view
 alter view $DATABASE.parquet_view rename to $DATABASE.new_parquet_view
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Test that the view was renamed
@@ -222,6 +235,7 @@ alter view $DATABASE.new_parquet_view as
 select bigint_col, string_col from functional_parquet.alltypesagg
 where bigint_col is null limit 10
 ---- RESULTS
+'View has been altered.'
 ====
 ---- QUERY
 # Test querying the altered view
@@ -236,6 +250,7 @@ bigint,bigint
 create view $DATABASE.const_view
 as select 1, 'a', cast(10.0 as float)
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 select * from $DATABASE.const_view
@@ -251,6 +266,7 @@ create view $DATABASE.paren_view as
 select count(*) from functional.alltypessmall
 where true and (true or false) and false
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that parentheses are preserved in view creation.
@@ -265,6 +281,7 @@ bigint
 create view $DATABASE.decimal_view as
 select * from functional.decimal_tbl
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Query a view with decimal columns. Regression test for IMPALA-1021.


[4/6] impala git commit: IMPALA-6820: Remove _impala_builtins from catalogd

Posted by ta...@apache.org.
IMPALA-6820: Remove _impala_builtins from catalogd

The _impala_builtins database is initialized in the constructor of
Catalog and hence is inherited by both the CatalogServiceCatalog and
ImpaladCatalog. Since  _impala_builtins is not used by the catalog
server and to avoid the overhead of managing this database during normal
metadata operations (e.g. invalidate), it is moved to the ImpaladCatalog
class.

Change-Id: I166d8086db1d2920408f38dc56fe7c70a4c143a8
Reviewed-on: http://gerrit.cloudera.org:8080/9947
Reviewed-by: Tianyi Wang <tw...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: d28b39afae5b8f1d621dcd2e1884c0353f1c058f
Parents: 2ee914d
Author: Tianyi Wang <tw...@cloudera.com>
Authored: Wed Apr 4 15:25:27 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Apr 11 03:55:07 2018 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/impala/analysis/CastExpr.java | 13 ++++++++-----
 fe/src/main/java/org/apache/impala/analysis/Expr.java  |  3 ++-
 .../org/apache/impala/analysis/ExtractFromExpr.java    |  3 ++-
 .../org/apache/impala/analysis/FunctionCallExpr.java   |  3 ++-
 .../java/org/apache/impala/analysis/FunctionName.java  |  8 +++++---
 .../main/java/org/apache/impala/catalog/Catalog.java   | 12 ------------
 .../java/org/apache/impala/catalog/ImpaladCatalog.java |  8 ++++++++
 .../java/org/apache/impala/catalog/ScalarFunction.java |  4 ++--
 fe/src/main/jflex/sql-scanner.flex                     |  2 +-
 .../org/apache/impala/analysis/AnalyzeExprsTest.java   |  7 ++++---
 10 files changed, 34 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CastExpr.java b/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
index 29e1736..c9a8511 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
@@ -21,6 +21,7 @@ import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.Function.CompareMode;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.ScalarType;
@@ -258,18 +259,20 @@ public class CastExpr extends Expr {
     noOp_ = childType.equals(type_);
     if (noOp_) return;
 
-    FunctionName fnName = new FunctionName(Catalog.BUILTINS_DB, getFnName(type_));
+    FunctionName fnName = new FunctionName(ImpaladCatalog.BUILTINS_DB, getFnName(type_));
     Type[] args = { childType };
     Function searchDesc = new Function(fnName, args, Type.INVALID, false);
     if (isImplicit_) {
-      fn_ = Catalog.getBuiltin(searchDesc, CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
+      fn_ = ImpaladCatalog.getBuiltinsDb().getFunction(searchDesc,
+          CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
       Preconditions.checkState(fn_ != null);
     } else {
-      fn_ = Catalog.getBuiltin(searchDesc, CompareMode.IS_IDENTICAL);
+      fn_ = ImpaladCatalog.getBuiltinsDb().getFunction(searchDesc,
+          CompareMode.IS_IDENTICAL);
       if (fn_ == null) {
         // allow for promotion from CHAR to STRING; only if no exact match is found
-        fn_ = Catalog.getBuiltin(searchDesc.promoteCharsToStrings(),
-            CompareMode.IS_IDENTICAL);
+        fn_ =  ImpaladCatalog.getBuiltinsDb().getFunction(
+            searchDesc.promoteCharsToStrings(), CompareMode.IS_IDENTICAL);
       }
     }
     if (fn_ == null) {

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/analysis/Expr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/Expr.java b/fe/src/main/java/org/apache/impala/analysis/Expr.java
index e1fb5a8..ef53476 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -29,6 +29,7 @@ import org.apache.impala.analysis.BinaryPredicate.Operator;
 import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.Function.CompareMode;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
@@ -407,7 +408,7 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
    */
   protected Function getBuiltinFunction(Analyzer analyzer, String name,
       Type[] argTypes, CompareMode mode) throws AnalysisException {
-    FunctionName fnName = new FunctionName(Catalog.BUILTINS_DB, name);
+    FunctionName fnName = new FunctionName(ImpaladCatalog.BUILTINS_DB, name);
     Function searchDesc = new Function(fnName, argTypes, Type.INVALID, false);
     return analyzer.getCatalog().getFunction(searchDesc, mode);
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
index 5e43f9f..eea48f8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
@@ -20,6 +20,7 @@ package org.apache.impala.analysis;
 import java.util.Set;
 
 import org.apache.impala.catalog.Catalog;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TExtractField;
@@ -73,7 +74,7 @@ public class ExtractFromExpr extends FunctionCallExpr {
           + " does not accept the keyword FROM.");
     }
     if ((getFnName().getDb() != null)
-        && !getFnName().getDb().equals(Catalog.BUILTINS_DB)) {
+        && !getFnName().getDb().equals(ImpaladCatalog.BUILTINS_DB)) {
       throw new AnalysisException("Function " + getFnName().toString() + " conflicts " +
           "with the EXTRACT builtin.");
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
index 12a34f7..46000b3 100644
--- a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
@@ -24,6 +24,7 @@ import org.apache.impala.catalog.AggregateFunction;
 import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.Function;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
@@ -111,7 +112,7 @@ public class FunctionCallExpr extends Expr {
     return fnName.getFnNamePath().size() == 1
            && fnName.getFnNamePath().get(0).equalsIgnoreCase(name)
         || fnName.getFnNamePath().size() == 2
-           && fnName.getFnNamePath().get(0).equals(Catalog.BUILTINS_DB)
+           && fnName.getFnNamePath().get(0).equals(ImpaladCatalog.BUILTINS_DB)
            && fnName.getFnNamePath().get(1).equalsIgnoreCase(name);
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/analysis/FunctionName.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/FunctionName.java b/fe/src/main/java/org/apache/impala/analysis/FunctionName.java
index 0ce8735..3ec4e63 100644
--- a/fe/src/main/java/org/apache/impala/analysis/FunctionName.java
+++ b/fe/src/main/java/org/apache/impala/analysis/FunctionName.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 
 import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.Db;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TFunctionName;
 import com.google.common.base.Joiner;
@@ -118,15 +119,16 @@ public class FunctionName {
     }
 
     // Resolve the database for this function.
-    Db builtinDb = analyzer.getCatalog().getBuiltinsDb();
+    Db builtinDb = ImpaladCatalog.getBuiltinsDb();
     if (!isFullyQualified()) {
       db_ = analyzer.getDefaultDb();
       if (preferBuiltinsDb && builtinDb.containsFunction(fn_)) {
-        db_ = Catalog.BUILTINS_DB;
+        db_ = ImpaladCatalog.BUILTINS_DB;
       }
     }
     Preconditions.checkNotNull(db_);
-    isBuiltin_ = db_.equals(Catalog.BUILTINS_DB) && builtinDb.containsFunction(fn_);
+    isBuiltin_ = db_.equals(ImpaladCatalog.BUILTINS_DB) &&
+        builtinDb.containsFunction(fn_);
     isAnalyzed_ = true;
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/catalog/Catalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Catalog.java b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
index 6f9cdb5..aba88df 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -59,7 +59,6 @@ public abstract class Catalog {
   // Initial catalog version.
   public final static long INITIAL_CATALOG_VERSION = 0L;
   public static final String DEFAULT_DB = "default";
-  public static final String BUILTINS_DB = "_impala_builtins";
 
   protected final MetaStoreClientPool metaStoreClientPool_ =
       new MetaStoreClientPool(0, 0);
@@ -75,9 +74,6 @@ public abstract class Catalog {
       new AtomicReference<ConcurrentHashMap<String, Db>>(
           new ConcurrentHashMap<String, Db>());
 
-  // DB that contains all builtins
-  private static Db builtinsDb_;
-
   // Cache of data sources.
   protected final CatalogObjectCache<DataSource> dataSources_;
 
@@ -88,8 +84,6 @@ public abstract class Catalog {
 
   public Catalog() {
     dataSources_ = new CatalogObjectCache<DataSource>();
-    builtinsDb_ = new BuiltinsDb(BUILTINS_DB);
-    addDb(builtinsDb_);
   }
 
   /**
@@ -104,8 +98,6 @@ public abstract class Catalog {
     metaStoreClientPool_.initClients(numClients, initialCnxnTimeoutSec);
   }
 
-  public Db getBuiltinsDb() { return builtinsDb_; }
-
   /**
    * Adds a new database to the catalog, replacing any existing database with the same
    * name. Returns the previous database with this name, or null if there was no
@@ -288,10 +280,6 @@ public abstract class Catalog {
     return db.getFunction(desc, mode);
   }
 
-  public static Function getBuiltin(Function desc, Function.CompareMode mode) {
-    return builtinsDb_.getFunction(desc, mode);
-  }
-
   /**
    * Removes a function from the catalog. Increments the catalog version and returns
    * the Function object that was removed if the function existed, otherwise returns

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
index fce4574..6ad4234 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
@@ -76,6 +76,7 @@ import com.google.common.base.Preconditions;
 public class ImpaladCatalog extends Catalog {
   private static final Logger LOG = Logger.getLogger(ImpaladCatalog.class);
   private static final TUniqueId INITIAL_CATALOG_SERVICE_ID = new TUniqueId(0L, 0L);
+  public static final String BUILTINS_DB = "_impala_builtins";
 
   // The last known Catalog Service ID. If the ID changes, it indicates the CatalogServer
   // has restarted.
@@ -98,8 +99,13 @@ public class ImpaladCatalog extends Catalog {
   // Used during table creation.
   private final String defaultKuduMasterHosts_;
 
+  // DB that contains all builtins
+  private static Db builtinsDb_;
+
   public ImpaladCatalog(String defaultKuduMasterHosts) {
     super();
+    builtinsDb_ = new BuiltinsDb(BUILTINS_DB);
+    addDb(builtinsDb_);
     defaultKuduMasterHosts_ = defaultKuduMasterHosts;
     // Ensure the contents of the CatalogObjectVersionQueue instance are cleared when a
     // new instance of ImpaladCatalog is created (see IMPALA-6486).
@@ -536,4 +542,6 @@ public class ImpaladCatalog extends Catalog {
     }
   }
   public TUniqueId getCatalogServiceId() { return catalogServiceId_; }
+
+  public static Db getBuiltinsDb() { return builtinsDb_; }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java b/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
index 8872e1e..5035d38 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
@@ -72,7 +72,7 @@ public class ScalarFunction extends Function {
       String prepareFnSymbol, String closeFnSymbol, boolean userVisible) {
     Preconditions.checkNotNull(symbol);
     ScalarFunction fn = new ScalarFunction(
-        new FunctionName(Catalog.BUILTINS_DB, name), argTypes, retType, hasVarArgs);
+        new FunctionName(ImpaladCatalog.BUILTINS_DB, name), argTypes, retType, hasVarArgs);
     fn.setBinaryType(TFunctionBinaryType.BUILTIN);
     fn.setUserVisible(userVisible);
     fn.setIsPersistent(true);
@@ -221,7 +221,7 @@ public class ScalarFunction extends Function {
       ArrayList<Type> argTypes, boolean hasVarArgs, Type retType,
       boolean userVisible) {
     ScalarFunction fn = new ScalarFunction(
-        new FunctionName(Catalog.BUILTINS_DB, name), argTypes, retType, hasVarArgs);
+        new FunctionName(ImpaladCatalog.BUILTINS_DB, name), argTypes, retType, hasVarArgs);
     fn.setBinaryType(TFunctionBinaryType.BUILTIN);
     fn.setUserVisible(userVisible);
     fn.setIsPersistent(true);

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/main/jflex/sql-scanner.flex
----------------------------------------------------------------------
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index c4ed217..dd1da7c 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -30,7 +30,7 @@ import java.util.HashSet;
 
 import org.apache.impala.analysis.SqlParserSymbols;
 import org.apache.impala.catalog.BuiltinsDb;
-import static org.apache.impala.catalog.Catalog.BUILTINS_DB;
+import static org.apache.impala.catalog.ImpaladCatalog.BUILTINS_DB;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TReservedWordsVersion;
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d28b39af/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
index cc71438..df99a62 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -34,6 +34,7 @@ import org.apache.impala.catalog.Column;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.Function.CompareMode;
+import org.apache.impala.catalog.ImpaladCatalog;
 import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.ScalarType;
@@ -2690,7 +2691,7 @@ public class AnalyzeExprsTest extends AnalyzerTest {
   @Test
   // IMPALA-2233: Regression test for loss of precision through implicit casts.
   public void TestImplicitArgumentCasts() throws AnalysisException {
-    FunctionName fnName = new FunctionName(Catalog.BUILTINS_DB, "greatest");
+    FunctionName fnName = new FunctionName(ImpaladCatalog.BUILTINS_DB, "greatest");
     Function tinyIntFn = new Function(fnName, new Type[] {ScalarType.DOUBLE},
         Type.DOUBLE, true);
     Function decimalFn = new Function(fnName,
@@ -2700,7 +2701,7 @@ public class AnalyzeExprsTest extends AnalyzerTest {
     Assert.assertTrue(tinyIntFn.compare(decimalFn,
         CompareMode.IS_NONSTRICT_SUPERTYPE_OF));
     // Check that this resolves to the decimal version of the function.
-    Db db = catalog_.getDb(Catalog.BUILTINS_DB);
+    Db db = catalog_.getDb(ImpaladCatalog.BUILTINS_DB);
     Function foundFn = db.getFunction(decimalFn, CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
     assertNotNull(foundFn);
     Assert.assertTrue(foundFn.getArgs()[0].isDecimal());
@@ -2733,7 +2734,7 @@ public class AnalyzeExprsTest extends AnalyzerTest {
     Assert.assertNotNull(foundFn);
     Assert.assertEquals(Type.DOUBLE, foundFn.getArgs()[0]);
 
-    FunctionName lagFnName = new FunctionName(Catalog.BUILTINS_DB, "lag");
+    FunctionName lagFnName = new FunctionName(ImpaladCatalog.BUILTINS_DB, "lag");
     // Timestamp should not be converted to string if string overload available.
     Function lagStringFn = new Function(lagFnName,
         new Type[] {ScalarType.STRING, Type.TINYINT}, Type.INVALID, false);


[5/6] impala git commit: IMPALA-5717: Support for reading ORC data files

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/common/thrift/CatalogObjects.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index ecd27dc..0f71f5f 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -58,7 +58,8 @@ enum THdfsFileFormat {
   SEQUENCE_FILE,
   AVRO,
   PARQUET,
-  KUDU
+  KUDU,
+  ORC
 }
 
 // TODO: Since compression is also enabled for Kudu columns, we should
@@ -73,7 +74,8 @@ enum THdfsCompression {
   SNAPPY_BLOCKED,
   LZO,
   LZ4,
-  ZLIB
+  ZLIB,
+  ZSTD
 }
 
 enum TColumnEncoding {

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 999ed16..f2d7cef 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -263,8 +263,8 @@ terminal
   KW_IN, KW_INCREMENTAL, KW_INIT_FN, KW_INNER, KW_INPATH, KW_INSERT, KW_INT,
   KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP, KW_IS, KW_JOIN,
   KW_KUDU, KW_LAST, KW_LEFT, KW_LIKE, KW_LIMIT, KW_LINES, KW_LOAD, KW_LOCATION, KW_MAP,
-  KW_MERGE_FN, KW_METADATA, KW_NOT, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORDER,
-  KW_OUTER, KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION,
+  KW_MERGE_FN, KW_METADATA, KW_NOT, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORC,
+  KW_ORDER, KW_OUTER, KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION,
   KW_PARTITIONED, KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED,
   KW_PURGE, KW_RANGE, KW_RCFILE, KW_RECOVER, KW_REFRESH, KW_REGEXP, KW_RENAME,
   KW_REPEATABLE, KW_REPLACE, KW_REPLICATION, KW_RESTRICT, KW_RETURNS, KW_REVOKE,
@@ -1562,6 +1562,8 @@ file_format_val ::=
   {: RESULT = THdfsFileFormat.PARQUET; :}
   | KW_PARQUETFILE
   {: RESULT = THdfsFileFormat.PARQUET; :}
+  | KW_ORC
+  {: RESULT = THdfsFileFormat.ORC; :}
   | KW_TEXTFILE
   {: RESULT = THdfsFileFormat.TEXT; :}
   | KW_SEQUENCEFILE
@@ -3487,6 +3489,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_OR:r
   {: RESULT = r.toString(); :}
+  | KW_ORC:r
+  {: RESULT = r.toString(); :}
   | KW_ORDER:r
   {: RESULT = r.toString(); :}
   | KW_OUTER:r

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java b/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
index 5df3dfa..e442d66 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
@@ -743,10 +743,10 @@ public class ComputeStatsStmt extends StatementBase {
   public Set<Column> getValidatedColumnWhitelist() { return validatedColumnWhitelist_; }
 
   /**
-   * Returns true if this statement computes stats on Parquet partitions only,
+   * Returns true if this statement computes stats on Parquet/ORC partitions only,
    * false otherwise.
    */
-  public boolean isParquetOnly() {
+  public boolean isColumnar() {
     if (!(table_ instanceof HdfsTable)) return false;
     Collection<HdfsPartition> affectedPartitions = null;
     if (partitionSet_ != null) {
@@ -755,7 +755,9 @@ public class ComputeStatsStmt extends StatementBase {
       affectedPartitions = ((HdfsTable) table_).getPartitions();
     }
     for (HdfsPartition partition: affectedPartitions) {
-      if (partition.getFileFormat() != HdfsFileFormat.PARQUET) return false;
+      if (partition.getFileFormat() != HdfsFileFormat.PARQUET
+          && partition.getFileFormat() != HdfsFileFormat.ORC)
+        return false;
     }
     return true;
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
index e4fce60..32cae72 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
@@ -62,6 +62,10 @@ public enum HdfsFileFormat {
       "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
       "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
       true, true),
+  ORC("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
+      "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
+      "org.apache.hadoop.hive.ql.io.orc.OrcSerde",
+      false, true),
   KUDU("org.apache.kudu.mapreduce.KuduTableInputFormat",
       "org.apache.kudu.mapreduce.KuduTableOutputFormat",
       "", false, false);
@@ -99,19 +103,23 @@ public enum HdfsFileFormat {
       "parquet.hive.MapredParquetInputFormat"
   };
 
-  private static final Map<String, HdfsFileFormat> VALID_INPUT_FORMATS =
-      ImmutableMap.<String, HdfsFileFormat>builder()
-          .put(RC_FILE.inputFormat(), RC_FILE)
-          .put(TEXT.inputFormat(), TEXT)
-          .put(LZO_TEXT.inputFormat(), TEXT)
-          .put(SEQUENCE_FILE.inputFormat(), SEQUENCE_FILE)
-          .put(AVRO.inputFormat(), AVRO)
-          .put(PARQUET.inputFormat(), PARQUET)
-          .put(PARQUET_LEGACY_INPUT_FORMATS[0], PARQUET)
-          .put(PARQUET_LEGACY_INPUT_FORMATS[1], PARQUET)
-          .put(PARQUET_LEGACY_INPUT_FORMATS[2], PARQUET)
-          .put(KUDU.inputFormat(), KUDU)
-          .build();
+  private static Map<String, HdfsFileFormat> VALID_INPUT_FORMATS;
+  public static void init(boolean enableOrcScanner) {
+    ImmutableMap.Builder<String, HdfsFileFormat> builder =
+        ImmutableMap.<String, HdfsFileFormat>builder()
+            .put(RC_FILE.inputFormat(), RC_FILE)
+            .put(TEXT.inputFormat(), TEXT)
+            .put(LZO_TEXT.inputFormat(), TEXT)
+            .put(SEQUENCE_FILE.inputFormat(), SEQUENCE_FILE)
+            .put(AVRO.inputFormat(), AVRO)
+            .put(PARQUET.inputFormat(), PARQUET)
+            .put(PARQUET_LEGACY_INPUT_FORMATS[0], PARQUET)
+            .put(PARQUET_LEGACY_INPUT_FORMATS[1], PARQUET)
+            .put(PARQUET_LEGACY_INPUT_FORMATS[2], PARQUET)
+            .put(KUDU.inputFormat(), KUDU);
+    if (enableOrcScanner) builder.put(ORC.inputFormat(), ORC);
+    VALID_INPUT_FORMATS = builder.build();
+  }
 
   /**
    * Returns true if the string describes an input format class that we support.
@@ -145,6 +153,7 @@ public enum HdfsFileFormat {
       case TEXT: return HdfsFileFormat.TEXT;
       case SEQUENCE_FILE: return HdfsFileFormat.SEQUENCE_FILE;
       case AVRO: return HdfsFileFormat.AVRO;
+      case ORC: return HdfsFileFormat.ORC;
       case PARQUET: return HdfsFileFormat.PARQUET;
       case KUDU: return HdfsFileFormat.KUDU;
       default:
@@ -159,6 +168,7 @@ public enum HdfsFileFormat {
       case TEXT: return THdfsFileFormat.TEXT;
       case SEQUENCE_FILE: return THdfsFileFormat.SEQUENCE_FILE;
       case AVRO: return THdfsFileFormat.AVRO;
+      case ORC: return THdfsFileFormat.ORC;
       case PARQUET: return THdfsFileFormat.PARQUET;
       case KUDU: return THdfsFileFormat.KUDU;
       default:
@@ -170,6 +180,7 @@ public enum HdfsFileFormat {
   public String toSql(HdfsCompression compressionType) {
     switch (this) {
       case RC_FILE: return "RCFILE";
+      case ORC: return "ORC";
       case TEXT:
         if (compressionType == HdfsCompression.LZO ||
             compressionType == HdfsCompression.LZO_INDEX) {
@@ -240,6 +251,7 @@ public enum HdfsFileFormat {
       case SEQUENCE_FILE:
       case AVRO:
       case PARQUET:
+      case ORC:
         return true;
       case KUDU:
         return false;

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java b/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
index b4e2564..f51b10e 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
@@ -57,6 +57,7 @@ public class HdfsStorageDescriptor {
       "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", // (seq / text / parquet)
       "org.apache.hadoop.hive.serde2.avro.AvroSerDe", // (avro)
       "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", // (rc)
+      "org.apache.hadoop.hive.ql.io.orc.OrcSerde", // (orc)
       "parquet.hive.serde.ParquetHiveSerDe", // (parquet - legacy)
       // TODO: Verify the following Parquet SerDe works with Impala and add
       // support for the new input/output format classes. See IMPALA-4214.

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 7735f98..ac67d7d 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -332,11 +332,12 @@ public class HdfsScanNode extends ScanNode {
     Set<HdfsFileFormat> fileFormats = computeScanRangeLocations(analyzer);
 
     // Determine backend scan node implementation to use. The optimized MT implementation
-    // is currently only supported for Parquet.
+    // is currently supported for Parquet, ORC and Text.
     if (analyzer.getQueryOptions().isSetMt_dop() &&
         analyzer.getQueryOptions().mt_dop > 0 &&
         fileFormats.size() == 1 &&
         (fileFormats.contains(HdfsFileFormat.PARQUET)
+          || fileFormats.contains(HdfsFileFormat.ORC)
           || fileFormats.contains(HdfsFileFormat.TEXT))) {
       useMtScanNode_ = true;
     } else {
@@ -1191,9 +1192,10 @@ public class HdfsScanNode extends ScanNode {
     Preconditions.checkNotNull(desc_.getTable() instanceof HdfsTable);
     HdfsTable table = (HdfsTable) desc_.getTable();
     int perHostScanRanges;
-    if (table.getMajorityFormat() == HdfsFileFormat.PARQUET) {
+    if (table.getMajorityFormat() == HdfsFileFormat.PARQUET
+        || table.getMajorityFormat() == HdfsFileFormat.ORC) {
       // For the purpose of this estimation, the number of per-host scan ranges for
-      // Parquet files are equal to the number of columns read from the file. I.e.
+      // Parquet/ORC files are equal to the number of columns read from the file. I.e.
       // excluding partition columns and columns that are populated from file metadata.
       perHostScanRanges = 0;
       for (SlotDescriptor slot: desc_.getSlots()) {

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/service/BackendConfig.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 3833094..a94f46e 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.impala.analysis.SqlScanner;
+import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.thrift.TBackendGflags;
 
 import com.google.common.base.Preconditions;
@@ -45,6 +46,7 @@ public class BackendConfig {
     Preconditions.checkNotNull(cfg);
     INSTANCE = new BackendConfig(cfg);
     SqlScanner.init(cfg.getReserved_words_version());
+    HdfsFileFormat.init(cfg.isEnable_orc_scanner());
     initAuthToLocal();
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 392c249..348adaf 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -1014,11 +1014,11 @@ public class Frontend {
       if (thriftLineageGraph != null && thriftLineageGraph.isSetQuery_text()) {
         result.catalog_op_request.setLineage_graph(thriftLineageGraph);
       }
-      // Set MT_DOP=4 for COMPUTE STATS on Parquet tables, unless the user has already
+      // Set MT_DOP=4 for COMPUTE STATS on Parquet/ORC tables, unless the user has already
       // provided another value for MT_DOP.
       if (!queryOptions.isSetMt_dop() &&
           analysisResult.isComputeStatsStmt() &&
-          analysisResult.getComputeStatsStmt().isParquetOnly()) {
+          analysisResult.getComputeStatsStmt().isColumnar()) {
         queryOptions.setMt_dop(4);
       }
       // If unset, set MT_DOP to 0 to simplify the rest of the code.

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/fe/src/main/jflex/sql-scanner.flex
----------------------------------------------------------------------
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index dd1da7c..0512a2a 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -176,6 +176,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("on", SqlParserSymbols.KW_ON);
     keywordMap.put("||", SqlParserSymbols.KW_OR);
     keywordMap.put("or", SqlParserSymbols.KW_OR);
+    keywordMap.put("orc", SqlParserSymbols.KW_ORC);
     keywordMap.put("order", SqlParserSymbols.KW_ORDER);
     keywordMap.put("outer", SqlParserSymbols.KW_OUTER);
     keywordMap.put("over", SqlParserSymbols.KW_OVER);

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/LineItemMultiBlock/README.dox
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/README.dox b/testdata/LineItemMultiBlock/README.dox
index 7608067..1d6db46 100755
--- a/testdata/LineItemMultiBlock/README.dox
+++ b/testdata/LineItemMultiBlock/README.dox
@@ -1,6 +1,7 @@
 This file was created for:
 IMPALA-1881: Maximize data locality when scanning Parquet files with multiple row groups.
 IMPALA-2466: Add more tests to the HDFS parquet scanner.
+IMPALA-5717: Add tests for HDFS orc scanner.
 
 The table lineitem_multiblock is a single parquet file with:
  - A row group size of approximately 12 KB each.
@@ -31,3 +32,21 @@ blocks.
 
 'lineitem_multiblock_one_row_group' was created similarly but with a much higher
 'parquet.block.size' so that everything fit in one row group.
+
+----
+
+The orc files are created by the following hive queries:
+
+use functional_orc_def;
+
+set orc.stripe.size=1024;
+set orc.compress=ZLIB;
+create table lineitem_threeblocks like tpch.lineitem stored as orc;
+create table lineitem_sixblocks like tpch.lineitem stored as orc;
+insert overwrite table lineitem_threeblocks select * from tpch.lineitem limit 16000;
+insert overwrite table lineitem_sixblocks select * from tpch.lineitem limit 30000;
+
+set orc.stripe.size=67108864;
+create table lineitem_orc_multiblock_one_stripe like tpch.lineitem stored as orc;
+insert overwrite table lineitem_orc_multiblock_one_stripe select * from
+tpch.lineitem limit 16000;

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/LineItemMultiBlock/lineitem_orc_multiblock_one_stripe.orc
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/lineitem_orc_multiblock_one_stripe.orc b/testdata/LineItemMultiBlock/lineitem_orc_multiblock_one_stripe.orc
new file mode 100644
index 0000000..7dbbffb
Binary files /dev/null and b/testdata/LineItemMultiBlock/lineitem_orc_multiblock_one_stripe.orc differ

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/LineItemMultiBlock/lineitem_sixblocks.orc
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/lineitem_sixblocks.orc b/testdata/LineItemMultiBlock/lineitem_sixblocks.orc
new file mode 100644
index 0000000..5fa6cfa
Binary files /dev/null and b/testdata/LineItemMultiBlock/lineitem_sixblocks.orc differ

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/LineItemMultiBlock/lineitem_threeblocks.orc
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/lineitem_threeblocks.orc b/testdata/LineItemMultiBlock/lineitem_threeblocks.orc
new file mode 100644
index 0000000..9b12540
Binary files /dev/null and b/testdata/LineItemMultiBlock/lineitem_threeblocks.orc differ

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index 311029d..e50515b 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -154,6 +154,9 @@ function load-custom-schemas {
   hadoop fs -mkdir -p /test-warehouse/chars_formats_parquet/
   hadoop fs -put -f ${IMPALA_HOME}/testdata/data/chars-formats.parquet \
     /test-warehouse/chars_formats_parquet
+  hadoop fs -mkdir -p /test-warehouse/chars_formats_orc_def/
+  hadoop fs -put -f ${IMPALA_HOME}/testdata/data/chars-formats.orc \
+    /test-warehouse/chars_formats_orc_def
   hadoop fs -mkdir -p /test-warehouse/chars_formats_text/
   hadoop fs -put -f ${IMPALA_HOME}/testdata/data/chars-formats.txt \
     /test-warehouse/chars_formats_text

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/bin/generate-schema-statements.py
----------------------------------------------------------------------
diff --git a/testdata/bin/generate-schema-statements.py b/testdata/bin/generate-schema-statements.py
index 34c2084..3f730e6 100755
--- a/testdata/bin/generate-schema-statements.py
+++ b/testdata/bin/generate-schema-statements.py
@@ -128,6 +128,7 @@ FILE_FORMAT_MAP = {
   'text': 'TEXTFILE',
   'seq': 'SEQUENCEFILE',
   'rc': 'RCFILE',
+  'orc': 'ORC',
   'parquet': 'PARQUET',
   'text_lzo':
     "\nINPUTFORMAT 'com.hadoop.mapred.DeprecatedLzoTextInputFormat'" +
@@ -219,7 +220,7 @@ def build_table_template(file_format, columns, partition_columns, row_format,
     else:
       tblproperties["avro.schema.url"] = "hdfs://%s/%s/%s/{table_name}.json" \
         % (options.hdfs_namenode, options.hive_warehouse_dir, avro_schema_dir)
-  elif file_format in 'parquet':
+  elif file_format in ['parquet', 'orc']:  # columnar formats don't need row format
     row_format_stmt = str()
   elif file_format == 'kudu':
     # Use partitioned_by to set a trivial hash distribution
@@ -243,7 +244,7 @@ def build_table_template(file_format, columns, partition_columns, row_format,
     for table_property in table_properties.split("\n"):
       format_prop = table_property.split(":")
       if format_prop[0] == file_format:
-        key_val = format_prop[1].split("=");
+        key_val = format_prop[1].split("=")
         tblproperties[key_val[0]] = key_val[1]
 
   all_tblproperties = []
@@ -658,7 +659,7 @@ def generate_statements(output_name, test_vectors, sections,
             # that weren't already added to the table. So, for force reload, manually
             # delete the partition directories.
             output.create.append(("DFS -rm -R {data_path};").format(
-              data_path=data_path));
+              data_path=data_path))
           else:
             # If this is not a force reload use msck repair to add the partitions
             # into the table.

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/bin/run-hive-server.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-hive-server.sh b/testdata/bin/run-hive-server.sh
index 49d1de2..3b2c83d 100755
--- a/testdata/bin/run-hive-server.sh
+++ b/testdata/bin/run-hive-server.sh
@@ -73,9 +73,10 @@ ${CLUSTER_BIN}/wait-for-metastore.py --transport=${METASTORE_TRANSPORT}
 
 if [ ${ONLY_METASTORE} -eq 0 ]; then
   # Starts a HiveServer2 instance on the port specified by the HIVE_SERVER2_THRIFT_PORT
-  # environment variable.
+  # environment variable. HADOOP_HEAPSIZE should be set to at least 2048 to avoid OOM
+  # when loading ORC tables like widerow.
   if [[ $IMPALA_MINICLUSTER_PROFILE == 2 ]]; then
-    HADOOP_HEAPSIZE="512" hive --service hiveserver2 > ${LOGDIR}/hive-server2.out 2>&1 &
+    HADOOP_HEAPSIZE="2048" hive --service hiveserver2 > ${LOGDIR}/hive-server2.out 2>&1 &
   elif [[ $IMPALA_MINICLUSTER_PROFILE == 3 ]]; then
     HADOOP_CLIENT_OPTS="-Xmx2048m -Dhive.log.file=hive-server2.log" hive \
       --service hiveserver2 > ${LOGDIR}/hive-server2.out 2>&1 &

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/cluster/node_templates/common/etc/hadoop/conf/hdfs-site.xml.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/common/etc/hadoop/conf/hdfs-site.xml.tmpl b/testdata/cluster/node_templates/common/etc/hadoop/conf/hdfs-site.xml.tmpl
index f72dd97..c9ee70b 100644
--- a/testdata/cluster/node_templates/common/etc/hadoop/conf/hdfs-site.xml.tmpl
+++ b/testdata/cluster/node_templates/common/etc/hadoop/conf/hdfs-site.xml.tmpl
@@ -82,6 +82,12 @@
     <value>134217728</value>
   </property>
 
+  <!-- Decrease this so we can create mini test files across several blocks -->
+  <property>
+    <name>dfs.namenode.fs-limits.min-block-size</name>
+    <value>1024</value>
+  </property>
+
   <!-- Set the max cached memory to ~64kb. This must be less than ulimit -l -->
   <property>
     <name>dfs.datanode.max.locked.memory</name>

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/data/chars-formats.orc
----------------------------------------------------------------------
diff --git a/testdata/data/chars-formats.orc b/testdata/data/chars-formats.orc
new file mode 100644
index 0000000..625c2c8
Binary files /dev/null and b/testdata/data/chars-formats.orc differ

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/datasets/functional/functional_schema_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index cede525..a7a5eac 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -739,6 +739,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION(p=1) SELECT i
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION(p=2) SELECT id, named_struct("f1",string_col,"f2",int_col), array(1, 2, 3), map("k", cast(0 as bigint)) FROM functional.alltypestiny;
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION(p=3) SELECT id, named_struct("f1",string_col,"f2",int_col), array(1, 2, 3), map("k", cast(0 as bigint)) FROM functional.alltypestiny;
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION(p=4) SELECT id, named_struct("f1",string_col,"f2",int_col), array(1, 2, 3), map("k", cast(0 as bigint)) FROM functional.alltypestiny;
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION(p=5) SELECT id, named_struct("f1",string_col,"f2",int_col), array(1, 2, 3), map("k", cast(0 as bigint)) FROM functional.alltypestiny;
 -- The order of insertions and alterations is deliberately chose to work around a Hive
 -- bug where the format of an altered partition is reverted back to the original format after
 -- an insert. So we first do the insert, and then alter the format.
@@ -746,6 +747,7 @@ USE {db_name}{db_suffix};
 ALTER TABLE {table_name} PARTITION (p=2) SET FILEFORMAT PARQUET;
 ALTER TABLE {table_name} PARTITION (p=3) SET FILEFORMAT AVRO;
 ALTER TABLE {table_name} PARTITION (p=4) SET FILEFORMAT RCFILE;
+ALTER TABLE {table_name} PARTITION (p=5) SET FILEFORMAT ORC;
 USE default;
 ====
 ---- DATASET

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/datasets/functional/schema_constraints.csv
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index ef65b9a..baf0306 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -66,6 +66,7 @@ table_name:complextypes_fileformat, constraint:restrict_to, table_format:parquet
 table_name:complextypes_fileformat, constraint:restrict_to, table_format:avro/snap/block
 table_name:complextypes_fileformat, constraint:restrict_to, table_format:rc/snap/block
 table_name:complextypes_fileformat, constraint:restrict_to, table_format:seq/snap/block
+table_name:complextypes_fileformat, constraint:restrict_to, table_format:orc/def/block
 table_name:complextypes_multifileformat, constraint:restrict_to, table_format:text/none/none
 
 # TODO: Avro
@@ -134,6 +135,8 @@ table_name:decimal_tbl, constraint:restrict_to, table_format:parquet/none/none
 table_name:decimal_tiny, constraint:restrict_to, table_format:parquet/none/none
 table_name:decimal_tbl, constraint:restrict_to, table_format:kudu/none/none
 table_name:decimal_tiny, constraint:restrict_to, table_format:kudu/none/none
+table_name:decimal_tbl, constraint:restrict_to, table_format:orc/def/block
+table_name:decimal_tiny, constraint:restrict_to, table_format:orc/def/block
 
 table_name:avro_decimal_tbl, constraint:restrict_to, table_format:avro/snap/block
 

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
index 9c68c65..1e61b7d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
@@ -15,6 +15,38 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=227B
    predicates: !empty(t.a)
 ====
+# Complex types are not supported on ORC.
+select 1 from functional_orc_def.complextypes_fileformat t, t.a
+---- PLAN
+not implemented: Scan of table 't' in format 'ORC' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
+Complex types are supported for these file formats: PARQUET.
+====
+select s.f1 from functional_orc_def.complextypes_fileformat t, t.m
+---- PLAN
+not implemented: Scan of table 't' in format 'ORC' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
+Complex types are supported for these file formats: PARQUET.
+====
+# Complex types are not supported on ORC, however queries materializing
+# only scalar type columns are allowed.
+select id from functional_orc_def.complextypes_fileformat
+---- PLAN
+PLAN-ROOT SINK
+|
+00:SCAN HDFS [functional_orc_def.complextypes_fileformat]
+   partitions=1/1 files=1 size=624B
+====
+# Complex types are not supported on ORC but count(*) and similar
+# queries should work.
+select count(*) from functional_orc_def.complextypes_fileformat
+---- PLAN
+PLAN-ROOT SINK
+|
+01:AGGREGATE [FINALIZE]
+|  output: count(*)
+|
+00:SCAN HDFS [functional_orc_def.complextypes_fileformat]
+   partitions=1/1 files=1 size=624B
+====
 # Complex types are not supported on Avro.
 select s.f1 from functional_avro_snap.complextypes_fileformat t, t.a
 ---- PLAN
@@ -111,11 +143,12 @@ select complex_struct_col.f1 from functional_hbase.allcomplextypes
 not implemented: Scan of table 'functional_hbase.allcomplextypes.complex_struct_col.f1' is not supported because 'functional_hbase.allcomplextypes' references a nested field/collection.
 Complex types are supported for these file formats: PARQUET.
 ====
-# The complextypes_multifileformat has three partitions with different file formats:
+# The complextypes_multifileformat has five partitions with different file formats:
 # p=1 text
 # p=2 parquet
 # p=3 avro
 # p=4 rc
+# p=5 orc
 # Scanning a text partition of a multi-format table with complex types fails.
 select 1 from functional.complextypes_multifileformat where p = 1
 ---- PLAN
@@ -136,7 +169,7 @@ PLAN-ROOT SINK
 |  03:UNNEST [t.a]
 |
 00:SCAN HDFS [functional.complextypes_multifileformat t]
-   partitions=1/4 files=1 size=128B
+   partitions=1/5 files=1 size=128B
    predicates: !empty(t.a)
 ====
 # Scanning an Avro partition of a multi-format table with complex types fails.
@@ -161,5 +194,23 @@ PLAN-ROOT SINK
 |  output: count(*)
 |
 00:SCAN HDFS [functional.complextypes_multifileformat]
-   partitions=1/4 files=1 size=128B
+   partitions=1/5 files=1 size=128B
+====
+# Scanning an ORC file partition of a multi-format table with complex types fails.
+select id from functional.complextypes_multifileformat t, t.a where p = 5
+---- PLAN
+not implemented: Scan of partition 'p=5' in format 'ORC' of table 't' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
+Complex types are supported for these file formats: PARQUET.
+====
+# Complex types are not supported on ORC files but count(*) and similar
+# queries should work.
+select count(*) from functional.complextypes_multifileformat where p = 5
+---- PLAN
+PLAN-ROOT SINK
+|
+01:AGGREGATE [FINALIZE]
+|  output: count(*)
+|
+00:SCAN HDFS [functional.complextypes_multifileformat]
+   partitions=1/5 files=1 size=128B
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-query/functional-query_core.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/functional-query_core.csv b/testdata/workloads/functional-query/functional-query_core.csv
index dffca78..7118e3f 100644
--- a/testdata/workloads/functional-query/functional-query_core.csv
+++ b/testdata/workloads/functional-query/functional-query_core.csv
@@ -2,6 +2,7 @@
 file_format:text, dataset:functional, compression_codec:none, compression_type:none
 file_format:seq, dataset:functional, compression_codec:snap, compression_type:block
 file_format:rc, dataset: functional, compression_codec: snap, compression_type: block
+file_format:orc, dataset: functional, compression_codec: def, compression_type: block
 file_format:parquet, dataset: functional, compression_codec: none, compression_type: none
 file_format:avro, dataset: functional, compression_codec: snap, compression_type: block
 file_format:hbase, dataset:functional, compression_codec:none, compression_type:none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-query/functional-query_dimensions.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/functional-query_dimensions.csv b/testdata/workloads/functional-query/functional-query_dimensions.csv
index 539122b..bcb4406 100644
--- a/testdata/workloads/functional-query/functional-query_dimensions.csv
+++ b/testdata/workloads/functional-query/functional-query_dimensions.csv
@@ -1,4 +1,4 @@
-file_format: text,seq,rc,avro,parquet,hbase,kudu
+file_format: text,seq,rc,avro,parquet,orc,hbase,kudu
 dataset: functional
 compression_codec: none,def,gzip,bzip,snap,lzo
 compression_type: none,block,record

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-query/functional-query_exhaustive.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/functional-query_exhaustive.csv b/testdata/workloads/functional-query/functional-query_exhaustive.csv
index 18331c6..a06ab52 100644
--- a/testdata/workloads/functional-query/functional-query_exhaustive.csv
+++ b/testdata/workloads/functional-query/functional-query_exhaustive.csv
@@ -22,5 +22,6 @@ file_format: avro, dataset: functional, compression_codec: none, compression_typ
 file_format: avro, dataset: functional, compression_codec: def, compression_type: block
 file_format: avro, dataset: functional, compression_codec: snap, compression_type: block
 file_format: parquet, dataset: functional, compression_codec: none, compression_type: none
+file_format: orc, dataset: functional, compression_codec: def, compression_type: block
 file_format: hbase, dataset: functional, compression_codec: none, compression_type: none
 file_format: kudu, dataset: functional, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-query/functional-query_pairwise.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/functional-query_pairwise.csv b/testdata/workloads/functional-query/functional-query_pairwise.csv
index 0a4ee09..e046a09 100644
--- a/testdata/workloads/functional-query/functional-query_pairwise.csv
+++ b/testdata/workloads/functional-query/functional-query_pairwise.csv
@@ -4,5 +4,6 @@ file_format: seq, dataset: functional, compression_codec: def, compression_type:
 file_format: rc, dataset: functional, compression_codec: gzip, compression_type: block
 file_format: avro, dataset: functional, compression_codec: snap, compression_type: block
 file_format: parquet, dataset: functional, compression_codec: none, compression_type: none
+file_format: orc, dataset: functional, compression_codec: def, compression_type: block
 file_format: hbase, dataset: functional, compression_codec: none, compression_type: none
 file_format: kudu, dataset: functional, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/functional-query/queries/DataErrorsTest/orc-type-checks.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/DataErrorsTest/orc-type-checks.test b/testdata/workloads/functional-query/queries/DataErrorsTest/orc-type-checks.test
new file mode 100644
index 0000000..ee06258
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/DataErrorsTest/orc-type-checks.test
@@ -0,0 +1,127 @@
+====
+---- QUERY
+select c1 from illtypes
+---- CATCH
+Type mismatch: table column BOOLEAN is map to column int in ORC file
+====
+---- QUERY
+select c2 from illtypes
+---- CATCH
+Type mismatch: table column FLOAT is map to column boolean in ORC file
+====
+---- QUERY
+select c3 from illtypes
+---- CATCH
+Type mismatch: table column BOOLEAN is map to column tinyint in ORC file
+====
+---- QUERY
+select c4 from illtypes
+---- CATCH
+Type mismatch: table column TINYINT is map to column smallint in ORC file
+====
+---- QUERY
+select c5 from illtypes
+---- CATCH
+Type mismatch: table column SMALLINT is map to column int in ORC file
+====
+---- QUERY
+select c6 from illtypes
+---- CATCH
+Type mismatch: table column INT is map to column bigint in ORC file
+====
+---- QUERY
+select c7 from illtypes
+---- CATCH
+Type mismatch: table column BOOLEAN is map to column float in ORC file
+====
+---- QUERY
+select c8 from illtypes
+---- CATCH
+Type mismatch: table column STRING is map to column double in ORC file
+====
+---- QUERY
+select c9 from illtypes
+---- CATCH
+Type mismatch: table column INT is map to column string in ORC file
+====
+---- QUERY
+select c10 from illtypes
+---- CATCH
+Type mismatch: table column FLOAT is map to column string in ORC file
+====
+---- QUERY
+select c11 from illtypes
+---- CATCH
+Type mismatch: table column BIGINT is map to column timestamp in ORC file
+====
+---- QUERY
+select * from safetypes order by c1
+---- TYPES
+bigint,boolean,smallint,int,bigint,bigint,double,double,char,string,timestamp,int,int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/','1',2009-02-01 00:01:00,2009,2
+4,true,0,0,0,0,0,0,'03/','0',2009-03-01 00:00:00,2009,3
+5,false,1,1,1,10,1.100000023841858,10.1,'03/','1',2009-03-01 00:01:00,2009,3
+6,true,0,0,0,0,0,0,'04/','0',2009-04-01 00:00:00,2009,4
+7,false,1,1,1,10,1.100000023841858,10.1,'04/','1',2009-04-01 00:01:00,2009,4
+====
+---- QUERY
+select d1 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+1234
+2345
+12345
+12345
+132842
+====
+---- QUERY
+select d2 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+---- CATCH
+It can't be truncated to table column DECIMAL(8,0) for column decimal(10,0) in ORC file
+====
+---- QUERY
+select d3 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+1.2345678900
+12.3456789000
+123.4567890000
+1234.5678900000
+12345.6789000000
+====
+---- QUERY
+select d4 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+---- CATCH
+Type mismatch: table column DECIMAL(20,20) is map to column decimal(38,38) in ORC file
+====
+---- QUERY
+select d5 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+---- CATCH
+Type mismatch: table column DECIMAL(2,0) is map to column decimal(10,5) in ORC file
+====
+---- QUERY
+select d6 from mismatch_decimals
+---- TYPES
+decimal
+---- RESULTS
+1
+1
+1
+1
+1
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpcds/tpcds_core.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/tpcds_core.csv b/testdata/workloads/tpcds/tpcds_core.csv
index 94b4b22..48cc97d 100644
--- a/testdata/workloads/tpcds/tpcds_core.csv
+++ b/testdata/workloads/tpcds/tpcds_core.csv
@@ -2,3 +2,4 @@
 file_format: text, dataset: tpcds, compression_codec: none, compression_type: none
 file_format: seq, dataset: tpcds, compression_codec: snap, compression_type: block
 file_format: parquet, dataset: tpcds, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpcds, compression_codec: def, compression_type: block

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpcds/tpcds_dimensions.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/tpcds_dimensions.csv b/testdata/workloads/tpcds/tpcds_dimensions.csv
index 8137b7a..bae5d90 100644
--- a/testdata/workloads/tpcds/tpcds_dimensions.csv
+++ b/testdata/workloads/tpcds/tpcds_dimensions.csv
@@ -1,4 +1,4 @@
-file_format: text,seq,rc,avro,parquet
+file_format: text,seq,rc,avro,parquet,orc
 dataset: tpcds
 compression_codec: none,def,gzip,bzip,snap,lzo
 compression_type: none,block,record

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpcds/tpcds_exhaustive.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/tpcds_exhaustive.csv b/testdata/workloads/tpcds/tpcds_exhaustive.csv
index c4b4f99..57fcddd 100644
--- a/testdata/workloads/tpcds/tpcds_exhaustive.csv
+++ b/testdata/workloads/tpcds/tpcds_exhaustive.csv
@@ -21,3 +21,6 @@ file_format: avro, dataset: tpcds, compression_codec: snap, compression_type: bl
 file_format: parquet, dataset: tpcds, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpcds, compression_codec: def, compression_type: block
 file_format: parquet, dataset: tpcds, compression_codec: snap, compression_type: block
+file_format: orc, dataset: tpcds, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpcds, compression_codec: def, compression_type: block
+file_format: orc, dataset: tpcds, compression_codec: snap, compression_type: block

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpcds/tpcds_pairwise.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/tpcds_pairwise.csv b/testdata/workloads/tpcds/tpcds_pairwise.csv
index e643495..61ee66c 100644
--- a/testdata/workloads/tpcds/tpcds_pairwise.csv
+++ b/testdata/workloads/tpcds/tpcds_pairwise.csv
@@ -13,3 +13,6 @@ file_format: rc, dataset: tpcds, compression_codec: def, compression_type: block
 file_format: avro, dataset: tpcds, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpcds, compression_codec: none, compression_type: none
 file_format: rc, dataset: tpcds, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpcds, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpcds, compression_codec: def, compression_type: block
+file_format: orc, dataset: tpcds, compression_codec: snap, compression_type: block

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpch/tpch_core.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_core.csv b/testdata/workloads/tpch/tpch_core.csv
index 86804ac..024063c 100644
--- a/testdata/workloads/tpch/tpch_core.csv
+++ b/testdata/workloads/tpch/tpch_core.csv
@@ -7,4 +7,5 @@ file_format:rc, dataset:tpch, compression_codec:none, compression_type:none
 file_format:avro, dataset:tpch, compression_codec: none, compression_type: none
 file_format:avro, dataset:tpch, compression_codec: snap, compression_type: block
 file_format:parquet, dataset:tpch, compression_codec: none, compression_type: none
+file_format:orc, dataset:tpch, compression_codec: def, compression_type: block
 file_format:kudu, dataset:tpch, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpch/tpch_dimensions.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_dimensions.csv b/testdata/workloads/tpch/tpch_dimensions.csv
index 1de34aa..f1ce5f0 100644
--- a/testdata/workloads/tpch/tpch_dimensions.csv
+++ b/testdata/workloads/tpch/tpch_dimensions.csv
@@ -1,4 +1,4 @@
-file_format: text,seq,rc,avro,parquet,kudu
+file_format: text,seq,rc,avro,parquet,orc,kudu
 dataset: tpch
 compression_codec: none,def,gzip,bzip,snap,lzo
 compression_type: none,block,record

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpch/tpch_exhaustive.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_exhaustive.csv b/testdata/workloads/tpch/tpch_exhaustive.csv
index 32085bf..3513dc5 100644
--- a/testdata/workloads/tpch/tpch_exhaustive.csv
+++ b/testdata/workloads/tpch/tpch_exhaustive.csv
@@ -22,4 +22,7 @@ file_format: avro, dataset: tpch, compression_codec: snap, compression_type: blo
 file_format: parquet, dataset: tpch, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpch, compression_codec: def, compression_type: block
 file_format: parquet, dataset: tpch, compression_codec: snap, compression_type: block
+file_format: orc, dataset: tpch, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpch, compression_codec: def, compression_type: block
+file_format: orc, dataset: tpch, compression_codec: snap, compression_type: block
 file_format: kudu, dataset:tpch, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/testdata/workloads/tpch/tpch_pairwise.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_pairwise.csv b/testdata/workloads/tpch/tpch_pairwise.csv
index 0744cf5..2eb4176 100644
--- a/testdata/workloads/tpch/tpch_pairwise.csv
+++ b/testdata/workloads/tpch/tpch_pairwise.csv
@@ -13,4 +13,7 @@ file_format: rc, dataset: tpch, compression_codec: def, compression_type: block
 file_format: avro, dataset: tpch, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpch, compression_codec: none, compression_type: none
 file_format: rc, dataset: tpch, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpch, compression_codec: none, compression_type: none
+file_format: orc, dataset: tpch, compression_codec: def, compression_type: block
+file_format: orc, dataset: tpch, compression_codec: snap, compression_type: block
 file_format: kudu, dataset:tpch, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/common/impala_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index d57b1cb..2e35c67 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -503,7 +503,7 @@ class ImpalaTestSuite(BaseTestSuite):
 
     Database names are dependent on the input format for table, which the table names
     remaining the same. A use database is issued before query execution. As such,
-    dabase names need to be build pre execution, this method wraps around the different
+    database names need to be build pre execution, this method wraps around the different
     execute methods and provides a common interface to issue the proper use command.
     """
     @wraps(function)

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/common/test_dimensions.py
----------------------------------------------------------------------
diff --git a/tests/common/test_dimensions.py b/tests/common/test_dimensions.py
index 4171e1f..df3f8c2 100644
--- a/tests/common/test_dimensions.py
+++ b/tests/common/test_dimensions.py
@@ -28,7 +28,7 @@ WORKLOAD_DIR = os.environ['IMPALA_WORKLOAD_DIR']
 # of what specific table format to target along with the exec options (num_nodes, etc)
 # to use when running the query.
 class TableFormatInfo(object):
-  KNOWN_FILE_FORMATS = ['text', 'seq', 'rc', 'parquet', 'avro', 'hbase']
+  KNOWN_FILE_FORMATS = ['text', 'seq', 'rc', 'parquet', 'orc', 'avro', 'hbase']
   if os.environ['KUDU_IS_SUPPORTED'] == 'true':
     KNOWN_FILE_FORMATS.append('kudu')
   KNOWN_COMPRESSION_CODECS = ['none', 'snap', 'gzip', 'bzip', 'def', 'lzo']

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/common/test_vector.py
----------------------------------------------------------------------
diff --git a/tests/common/test_vector.py b/tests/common/test_vector.py
index 4d22269..0c9cca4 100644
--- a/tests/common/test_vector.py
+++ b/tests/common/test_vector.py
@@ -52,7 +52,7 @@
 # otherwise. For example, if we want to make sure 'bool' columns are not used with 'sum':
 #
 # ImpalaTestMatrix.add_constraint(lambda v:\
-#    not (v.get_value('col_type') == 'bool and v.get_value('agg_func') == 'sum'))
+#    not (v.get_value('col_type') == 'bool' and v.get_value('agg_func') == 'sum'))
 #
 # Additional examples of usage can be found within the test suites.
 

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/comparison/cli_options.py
----------------------------------------------------------------------
diff --git a/tests/comparison/cli_options.py b/tests/comparison/cli_options.py
index 885ef84..1d737cf 100644
--- a/tests/comparison/cli_options.py
+++ b/tests/comparison/cli_options.py
@@ -221,7 +221,7 @@ def create_cluster(args):
 
 
 def add_storage_format_options(parser):
-  storage_formats = ['avro', 'parquet', 'rcfile', 'sequencefile', 'textfile']
+  storage_formats = ['avro', 'parquet', 'orc', 'rcfile', 'sequencefile', 'textfile']
   parser.add_argument(
       '--storage-file-formats', default=','.join(storage_formats),
       help='A comma separated list of storage formats to use.')

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/query_test/test_chars.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_chars.py b/tests/query_test/test_chars.py
index b182b91..4444410 100644
--- a/tests/query_test/test_chars.py
+++ b/tests/query_test/test_chars.py
@@ -57,6 +57,11 @@ class TestCharFormats(ImpalaTestSuite):
         STORED AS PARQUET
         LOCATION "{0}"'''.format(get_fs_path("/test-warehouse/chars_formats_parquet")))
     self.client.execute('''create external table if not exists
+        functional_orc_def.chars_formats
+        (cs CHAR(5), cl CHAR(140), vc VARCHAR(32))
+        STORED AS ORC
+        LOCATION "{0}"'''.format(get_fs_path("/test-warehouse/chars_formats_orc_def")))
+    self.client.execute('''create external table if not exists
         functional.chars_formats
         (cs CHAR(5), cl CHAR(140), vc VARCHAR(32))
         ROW FORMAT delimited fields terminated by ','  escaped by '\\\\'
@@ -84,6 +89,7 @@ class TestCharFormats(ImpalaTestSuite):
         (v.get_value('table_format').file_format in ['avro'] and
         v.get_value('table_format').compression_codec in ['snap']) or
         v.get_value('table_format').file_format in ['parquet'] or
+        v.get_value('table_format').file_format in ['orc'] or
         (v.get_value('table_format').file_format in ['text'] and
         v.get_value('table_format').compression_codec in ['none']))
 

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/query_test/test_decimal_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_decimal_queries.py b/tests/query_test/test_decimal_queries.py
index 3a14ed3..45a702d 100644
--- a/tests/query_test/test_decimal_queries.py
+++ b/tests/query_test/test_decimal_queries.py
@@ -43,7 +43,7 @@ class TestDecimalQueries(ImpalaTestSuite):
     cls.ImpalaTestMatrix.add_constraint(lambda v:\
         (v.get_value('table_format').file_format == 'text' and
          v.get_value('table_format').compression_codec == 'none') or
-         v.get_value('table_format').file_format in ['parquet', 'kudu'])
+         v.get_value('table_format').file_format in ['parquet', 'orc', 'kudu'])
 
   def test_queries(self, vector):
     self.run_test_case('QueryTest/decimal', vector)

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 2dcc213..bae52a6 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -105,7 +105,7 @@ class TestScannersAllTableFormatsWithLimit(ImpalaTestSuite):
     query_template = "select * from alltypes limit %s"
     for i in range(1, iterations):
       # Vary the limit to vary the timing of cancellation
-      limit = (iterations * 100) % 1000 + 1
+      limit = (i * 100) % 1001 + 1
       query = query_template % limit
       result = self.execute_query(query, vector.get_value('exec_option'),
           table_format=vector.get_value('table_format'))
@@ -837,7 +837,7 @@ class TestTextScanRangeLengths(ImpalaTestSuite):
 @SkipIfLocal.hive
 class TestScanTruncatedFiles(ImpalaTestSuite):
   @classmethod
-  def get_workload(self):
+  def get_workload(cls):
     return 'functional-query'
 
   @classmethod
@@ -900,3 +900,101 @@ class TestUncompressedText(ImpalaTestSuite):
     check_call(['hdfs', 'dfs', '-copyFromLocal', os.environ['IMPALA_HOME'] +
           "/testdata/data/lazy_timestamp.csv", tbl_loc])
     self.run_test_case('QueryTest/select-lazy-timestamp', vector, unique_database)
+
+class TestOrc(ImpalaTestSuite):
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestOrc, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_constraint(
+      lambda v: v.get_value('table_format').file_format == 'orc')
+
+  def test_misaligned_orc_stripes(self, vector, unique_database):
+    self._build_lineitem_table_helper(unique_database, 'lineitem_threeblocks',
+        'lineitem_threeblocks.orc')
+    self._build_lineitem_table_helper(unique_database, 'lineitem_sixblocks',
+        'lineitem_sixblocks.orc')
+    self._build_lineitem_table_helper(unique_database,
+        'lineitem_orc_multiblock_one_stripe',
+        'lineitem_orc_multiblock_one_stripe.orc')
+
+    # functional_orc.alltypes is well-formatted. 'NumScannersWithNoReads' counters are
+    # set to 0.
+    table_name = 'functional_orc_def.alltypes'
+    self._misaligned_orc_stripes_helper(table_name, 7300)
+    # lineitem_threeblock.orc is ill-formatted but every scanner reads some stripes.
+    # 'NumScannersWithNoReads' counters are set to 0.
+    table_name = unique_database + '.lineitem_threeblocks'
+    self._misaligned_orc_stripes_helper(table_name, 16000)
+    # lineitem_sixblocks.orc is ill-formatted but every scanner reads some stripes.
+    # 'NumScannersWithNoReads' counters are set to 0.
+    table_name = unique_database + '.lineitem_sixblocks'
+    self._misaligned_orc_stripes_helper(table_name, 30000)
+    # Scanning lineitem_orc_multiblock_one_stripe.orc finds two scan ranges that end up
+    # doing no reads because the file is poorly formatted.
+    table_name = unique_database + '.lineitem_orc_multiblock_one_stripe'
+    self._misaligned_orc_stripes_helper(
+      table_name, 16000, num_scanners_with_no_reads=2)
+
+  def _build_lineitem_table_helper(self, db, tbl, file):
+    self.client.execute("create table %s.%s like tpch.lineitem stored as orc" % (db, tbl))
+    tbl_loc = get_fs_path("/test-warehouse/%s.db/%s" % (db, tbl))
+    # set block size to 156672 so lineitem_threeblocks.orc occupies 3 blocks,
+    # lineitem_sixblocks.orc occupies 6 blocks.
+    check_call(['hdfs', 'dfs', '-Ddfs.block.size=156672', '-copyFromLocal',
+        os.environ['IMPALA_HOME'] + "/testdata/LineItemMultiBlock/" + file, tbl_loc])
+
+  def _misaligned_orc_stripes_helper(
+          self, table_name, rows_in_table, num_scanners_with_no_reads=0):
+    """Checks if 'num_scanners_with_no_reads' indicates the expected number of scanners
+    that don't read anything because the underlying file is poorly formatted
+    """
+    query = 'select * from %s' % table_name
+    result = self.client.execute(query)
+    assert len(result.data) == rows_in_table
+
+    runtime_profile = str(result.runtime_profile)
+    num_scanners_with_no_reads_list = re.findall(
+      'NumScannersWithNoReads: ([0-9]*)', runtime_profile)
+
+    # This will fail if the number of impalads != 3
+    # The fourth fragment is the "Averaged Fragment"
+    assert len(num_scanners_with_no_reads_list) == 4
+
+    # Calculate the total number of scan ranges that ended up not reading anything because
+    # an underlying file was poorly formatted.
+    # Skip the Averaged Fragment; it comes first in the runtime profile.
+    total = 0
+    for n in num_scanners_with_no_reads_list[1:]:
+      total += int(n)
+    assert total == num_scanners_with_no_reads
+
+  def test_type_conversions(self, vector, unique_database):
+    # Create an "illtypes" table whose columns can't match the underlining ORC file's.
+    # Create an "safetypes" table likes above but ORC columns can still fit into it.
+    # Reuse the data files of functional_orc_def.alltypestiny
+    tbl_loc = get_fs_path("/test-warehouse/alltypestiny_orc_def")
+    self.client.execute("""create external table %s.illtypes (c1 boolean, c2 float,
+        c3 boolean, c4 tinyint, c5 smallint, c6 int, c7 boolean, c8 string, c9 int,
+        c10 float, c11 bigint) partitioned by (year int, month int) stored as ORC
+        location '%s';""" % (unique_database, tbl_loc))
+    self.client.execute("""create external table %s.safetypes (c1 bigint, c2 boolean,
+        c3 smallint, c4 int, c5 bigint, c6 bigint, c7 double, c8 double, c9 char(3),
+        c10 varchar(3), c11 timestamp) partitioned by (year int, month int) stored as ORC
+        location '%s';""" % (unique_database, tbl_loc))
+    self.client.execute("alter table %s.illtypes recover partitions" % unique_database)
+    self.client.execute("alter table %s.safetypes recover partitions" % unique_database)
+
+    # Create a decimal table whose precisions don't match the underlining orc files.
+    # Reuse the data files of functional_orc_def.decimal_tbl.
+    decimal_loc = get_fs_path("/test-warehouse/decimal_tbl_orc_def")
+    self.client.execute("""create external table %s.mismatch_decimals (d1 decimal(8,0),
+        d2 decimal(8,0), d3 decimal(19,10), d4 decimal(20,20), d5 decimal(2,0))
+        partitioned by (d6 decimal(9,0)) stored as orc location '%s'"""
+        % (unique_database, decimal_loc))
+    self.client.execute("alter table %s.mismatch_decimals recover partitions" % unique_database)
+
+    self.run_test_case('DataErrorsTest/orc-type-checks', vector, unique_database)

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/query_test/test_scanners_fuzz.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners_fuzz.py b/tests/query_test/test_scanners_fuzz.py
index c336a17..791c343 100644
--- a/tests/query_test/test_scanners_fuzz.py
+++ b/tests/query_test/test_scanners_fuzz.py
@@ -61,6 +61,8 @@ class TestScannersFuzzing(ImpalaTestSuite):
           'num_nodes' : cls.NUM_NODES_VALUES,
           'mem_limit' : cls.MEM_LIMITS}))
     # TODO: enable for more table formats once they consistently pass the fuzz test.
+    # TODO(IMPALA-6772): enable for ORC formats once a new version after release-1.4.3
+    # of ORC library is released.
     cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ('avro', 'parquet') or
         (v.get_value('table_format').file_format == 'text' and

http://git-wip-us.apache.org/repos/asf/impala/blob/818cd8fa/tests/query_test/test_tpch_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_tpch_queries.py b/tests/query_test/test_tpch_queries.py
index ece8347..68a2984 100644
--- a/tests/query_test/test_tpch_queries.py
+++ b/tests/query_test/test_tpch_queries.py
@@ -36,7 +36,7 @@ class TestTpchQuery(ImpalaTestSuite):
     # TODO: the planner tests are based on text and need this.
     if cls.exploration_strategy() == 'core':
       cls.ImpalaTestMatrix.add_constraint(lambda v:\
-          v.get_value('table_format').file_format in ['text', 'parquet', 'kudu'])
+          v.get_value('table_format').file_format in ['text', 'parquet', 'kudu', 'orc'])
 
   def idfn(val):
     return "TPC-H: Q{0}".format(val)