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

[impala] branch master updated (95a1da2 -> ae82951)

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

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


    from 95a1da2  IMPALA-8578: part 2: move metrics code to .cc files
     new d9af995  IMPALA-8564: Add table/view create time in the lineage graph
     new e573b55  IMPALA-8562: Data cache should skip insertion of uncacheable ScanRanges
     new 51e8175  IMPALA-8450: Add support for zstd in parquet
     new f0678b0  IMPALA-7369: part 1: Implement TRUNC, DATE_TRUNC, EXTRACT, DATE_PART functions for DATE
     new 74a763c  IMPALA-8369: Fix HMS integration tests for Hive 3
     new ae82951  IMPALA-7957: Fix slot equivalences may be enforced multiple times

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CMakeLists.txt                                     |    5 +
 be/CMakeLists.txt                                  |    1 +
 be/src/benchmarks/CMakeLists.txt                   |    1 +
 be/src/benchmarks/date-benchmark.cc                |  118 ++
 be/src/catalog/catalog-util.cc                     |    4 +-
 be/src/exec/hdfs-table-sink.cc                     |    2 +-
 be/src/exec/parquet/hdfs-parquet-table-writer.cc   |  104 +-
 be/src/exec/parquet/parquet-common.cc              |   21 +-
 be/src/exec/parquet/parquet-metadata-utils.cc      |    3 +-
 be/src/experiments/compression-test.cc             |    7 +-
 be/src/exprs/expr-test.cc                          |  252 +++
 be/src/exprs/udf-builtins-ir.cc                    |  295 ++-
 be/src/exprs/udf-builtins.cc                       |  567 +++++-
 be/src/exprs/udf-builtins.h                        |  160 +-
 be/src/runtime/date-test.cc                        |   87 +
 be/src/runtime/date-value.cc                       |   94 +-
 be/src/runtime/date-value.h                        |   14 +
 be/src/runtime/io/data-cache-test.cc               |   18 +
 be/src/runtime/io/data-cache.cc                    |   14 +
 be/src/runtime/io/data-cache.h                     |   24 +-
 be/src/runtime/io/request-ranges.h                 |    2 +
 be/src/service/child-query.cc                      |   16 +-
 be/src/service/client-request-state.h              |    8 +-
 be/src/service/frontend.cc                         |    3 +-
 be/src/service/impala-server.cc                    |   16 +
 be/src/service/query-options-test.cc               |   55 +-
 be/src/service/query-options.cc                    |   53 +-
 be/src/util/codec.cc                               |   24 +-
 be/src/util/codec.h                                |   14 +-
 be/src/util/compress.cc                            |   24 +-
 be/src/util/compress.h                             |   18 +
 be/src/util/decompress-test.cc                     |   39 +-
 be/src/util/decompress.cc                          |   25 +-
 be/src/util/decompress.h                           |   12 +
 be/src/util/lineage-util.h                         |    9 +
 be/src/util/runtime-profile.cc                     |    4 +-
 bin/bootstrap_toolchain.py                         |    2 +-
 bin/impala-config.sh                               |    2 +
 cmake_modules/{FindRe2.cmake => FindZstd.cmake}    |   56 +-
 common/function-registry/impala_functions.py       |   42 +-
 common/thrift/CatalogObjects.thrift                |    8 +
 common/thrift/CatalogService.thrift                |    8 +
 common/thrift/ImpalaInternalService.thrift         |    2 +-
 common/thrift/LineageGraph.thrift                  |   15 +-
 common/thrift/generate_error_codes.py              |    2 +
 .../java/org/apache/impala/analysis/Analyzer.java  |   87 +-
 .../apache/impala/analysis/BinaryPredicate.java    |    9 +-
 .../apache/impala/analysis/ColumnLineageGraph.java |  665 ++++---
 .../impala/analysis/CreateOrAlterViewStmtBase.java |    5 +-
 .../apache/impala/analysis/ExtractFromExpr.java    |   32 +-
 .../org/apache/impala/analysis/SlotDescriptor.java |   11 +
 .../java/org/apache/impala/analysis/SlotRef.java   |    1 +
 .../impala/catalog/CatalogServiceCatalog.java      |    1 -
 .../java/org/apache/impala/planner/Planner.java    |   10 +-
 .../apache/impala/planner/SingleNodePlanner.java   |    7 +-
 .../apache/impala/service/CatalogOpExecutor.java   |    7 +-
 .../org/apache/impala/service/JniFrontend.java     |    1 -
 .../apache/impala/analysis/AnalyzeExprsTest.java   |   11 +-
 .../org/apache/impala/planner/PlannerTestBase.java |    8 +-
 .../queries/PlannerTest/inline-view.test           |  380 ++++
 .../queries/PlannerTest/lineage.test               | 2004 ++++++++++++++++----
 .../queries/QueryTest/hdfs-partitions.test         |    2 +-
 .../QueryTest/insert_parquet_multi_codecs.test     |   42 +
 .../functional-query/queries/QueryTest/set.test    |    4 +-
 tests/common/test_dimensions.py                    |    2 +-
 tests/custom_cluster/test_lineage.py               |   38 +-
 tests/metadata/test_hms_integration.py             |   67 +-
 tests/query_test/test_insert.py                    |    2 +-
 tests/query_test/test_insert_parquet.py            |   32 +-
 69 files changed, 4595 insertions(+), 1083 deletions(-)
 create mode 100644 be/src/benchmarks/date-benchmark.cc
 copy cmake_modules/{FindRe2.cmake => FindZstd.cmake} (53%)
 create mode 100644 testdata/workloads/functional-query/queries/QueryTest/insert_parquet_multi_codecs.test


[impala] 05/06: IMPALA-8369: Fix HMS integration tests for Hive 3

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 74a763c20d7e5b706350924ce8d71b517c7f5ca5
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
AuthorDate: Thu May 30 16:49:05 2019 +0200

    IMPALA-8369: Fix HMS integration tests for Hive 3
    
    Hive 3 changed the output format of the 'describe formatted
    <table> <column>' command. Added a new function that can parse
    the new output.
    
    Hive 3 also got more permissive during scanning, e.g. it can scan
    a Parquet column of INT type while the HMS metadata for that column
    is STRING. It does automatic conversion during scanning. However,
    it doesn't do the opposite, i.e. when the Parquet column type is
    STRING and the HMS column type is INT it raises an exception. Extended
    the test with this case.
    
    Change-Id: I51c91c7fc706175295b78abaacf47a86156714ce
    (cherry picked from commit 5fb83e73e83d5a6496553d88daa9a2510e5b97bd)
    Reviewed-on: http://gerrit.cloudera.org:8080/13475
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/metadata/test_hms_integration.py | 67 ++++++++++++++++++++++++++--------
 1 file changed, 52 insertions(+), 15 deletions(-)

diff --git a/tests/metadata/test_hms_integration.py b/tests/metadata/test_hms_integration.py
index c1a1734..4e7fdb6 100644
--- a/tests/metadata/test_hms_integration.py
+++ b/tests/metadata/test_hms_integration.py
@@ -29,7 +29,7 @@ import random
 import string
 from subprocess import call
 
-from tests.common.environ import IMPALA_TEST_CLUSTER_PROPERTIES
+from tests.common.environ import IMPALA_TEST_CLUSTER_PROPERTIES, HIVE_MAJOR_VERSION
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfS3, SkipIfABFS, SkipIfADLS, SkipIfIsilon, SkipIfLocal
 from tests.common.test_dimensions import (
@@ -190,11 +190,9 @@ class TestHmsIntegration(ImpalaTestSuite):
       result[stats[0]] = attributes
     return result
 
-  def hive_column_stats(self, table, column):
-    """Returns a dictionary of stats for a column according to Hive."""
-    output = self.run_stmt_in_hive(
-        'describe formatted %s %s' %
-        (table, column))
+  def parse_hive2_describe_formatted_output(self, output):
+    """Parses the output of a 'describe formatted' statement for Hive 2. Returns a
+    dictionary that holds the parsed attributes."""
     result = {}
     output_lines = output.split('\n')
     stat_names = map(string.strip, output_lines[0].split(','))
@@ -204,6 +202,24 @@ class TestHmsIntegration(ImpalaTestSuite):
       result[stat_names[i]] = stat_values[i]
     return result
 
+  def parse_hive3_describe_formatted_output(self, output):
+    """Parses the output of a 'describe formatted' statement for Hive 3. Returns a
+    dictionary that holds the parsed attributes."""
+    result = {}
+    for line in output.split('\n'):
+      line_elements = map(string.strip, line.split(','))
+      if len(line_elements) >= 2:
+        result[line_elements[0]] = line_elements[1]
+    return result
+
+  def hive_column_stats(self, table, column):
+    """Returns a dictionary of stats for a column according to Hive."""
+    output = self.run_stmt_in_hive('describe formatted %s %s' % (table, column))
+    if HIVE_MAJOR_VERSION == 2:
+      return self.parse_hive2_describe_formatted_output(output)
+    else:
+      return self.parse_hive3_describe_formatted_output(output)
+
   def impala_columns(self, table_name):
     """
     Returns a dict with column names as the keys and dicts of type and comments
@@ -595,7 +611,7 @@ class TestHmsIntegration(ImpalaTestSuite):
   @pytest.mark.execute_serially
   def test_change_parquet_column_type(self, vector):
     """
-    Changing column types in Parquet doesn't work in Hive and it causes
+    Changing column types in Parquet doesn't always work in Hive and it causes
     'select *' to fail in Impala as well, after invalidating metadata. This is a
     known issue with changing column types in Hive/parquet.
     """
@@ -603,23 +619,44 @@ class TestHmsIntegration(ImpalaTestSuite):
     with HiveDbWrapper(self, self.unique_string()) as db_name:
       with HiveTableWrapper(self, db_name + '.' + self.unique_string(),
                             '(x int, y int) stored as parquet') as table_name:
-        self.run_stmt_in_hive(
-            'insert into table %s values (33,44)' % table_name)
+        # The following INSERT statement creates a Parquet file with INT columns.
+        self.run_stmt_in_hive('insert into table %s values (33,44)' % table_name)
         assert '33,44' == self.run_stmt_in_hive(
             'select * from %s' % table_name).split('\n')[1]
         self.client.execute('invalidate metadata')
         assert '33\t44' == self.client.execute(
             'select * from %s' % table_name).get_data()
+        # Modify table metadata. After this statement, the table metadata in HMS
+        # and the Parquet file metadata won't agree on the type of column 'y'.
         self.run_stmt_in_hive('alter table %s change y y string' % table_name)
-        self.assert_sql_error(
-            self.run_stmt_in_hive, 'select * from %s' %
-            table_name, 'Cannot inspect org.apache.hadoop.io.IntWritable')
+        if HIVE_MAJOR_VERSION == 2:
+          # Hive 2 doesn't allow implicit conversion from INT to STRING.
+          self.assert_sql_error(
+              self.run_stmt_in_hive, 'select * from %s' % table_name,
+              'Cannot inspect org.apache.hadoop.io.IntWritable')
+        else:
+          # Hive 3 implicitly converts INTs to STRINGs.
+          assert '33,44' == self.run_stmt_in_hive(
+              'select * from %s' % table_name).split('\n')[1]
         self.client.execute('invalidate metadata %s' % table_name)
+        # Impala doesn't convert INTs to STRINGs implicitly.
         self.assert_sql_error(
-            self.client.execute,
-            'select * from %s' %
-            table_name,
+            self.client.execute, 'select * from %s' % table_name,
             "Column type: STRING, Parquet schema:")
+        # Insert STRING value, it will create a Parquet file where column 'y'
+        # has type STRING.
+        self.run_stmt_in_hive('insert into table %s values (33,\'100\')' % table_name)
+        # Modify HMS table metadata again, change the type of column 'y' back to INT.
+        self.run_stmt_in_hive('alter table %s change y y int' % table_name)
+        # Neither Hive 2 and 3, nor Impala converts STRINGs to INTs implicitly.
+        self.assert_sql_error(
+            self.run_stmt_in_hive, 'select * from %s' % table_name,
+            'org.apache.hadoop.io.Text cannot be '
+            'cast to org.apache.hadoop.io.IntWritable')
+        self.client.execute('invalidate metadata %s' % table_name)
+        self.assert_sql_error(
+            self.client.execute, 'select * from %s' % table_name,
+            "Column type: INT, Parquet schema:")
 
   @pytest.mark.execute_serially
   def test_change_table_name(self, vector):


[impala] 03/06: IMPALA-8450: Add support for zstd in parquet

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 51e8175c622014064e5a6853317de13b6987c629
Author: Abhishek <ar...@cloudera.com>
AuthorDate: Tue Jun 4 00:22:33 2019 -0700

    IMPALA-8450: Add support for zstd in parquet
    
    Makefile was updated to include zstd in the ${IMPALA_HOME}/toolchain
    directory. Other changes were made to make zstd headers and libs
    accessible.
    
    Class ZstandardCompressor/ZstandardDecompressor was added to provide
    interfaces for calling ZSTD_compress/ZSTD_decompress functions. Zstd
    supports different compression levels (clevel) from 1 to
    ZSTD_maxCLevel(). Zstd also supports -ive clevels, but since the -ive
    values represents uncompressed data they won't be supported. The default
    clevel is ZSTD_CLEVEL_DEFAULT.
    
    HdfsParquetTableWriter was updated to support ZSTD codec. The
    new codecs can be set using existing query option as follows:
      set COMPRESSION_CODEC=ZSTD:<clevel>;
      set COMPRESSION_CODEC=ZSTD; // uses ZSTD_CLEVEL_DEFAULT
    
    Testing:
      - Added unit test in DecompressorTest class with ZSTD_CLEVEL_DEFAULT
        clevel and a random clevel. The test unit decompresses an input
        compressed data and validates the result. It also tests for
        expected behavior when passing an over/under sized buffer for
        decompressing.
      - Added unit tests for valid/invalid values for COMPRESSION_CODEC.
      - Added e2e test in test_insert_parquet.py which tests writing/read-
        ing (null/non-null) data into/from a table (w different data type
        columns) using multiple codecs. Other existing e2e tests were
        updated to also use parquet/zstd table format.
      - Manual interoperability tests were run between Impala and Hive.
    
    Change-Id: Id2c0e26e6f7fb2dc4024309d733983ba5197beb7
    Reviewed-on: http://gerrit.cloudera.org:8080/13507
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 CMakeLists.txt                                     |   5 +
 be/CMakeLists.txt                                  |   1 +
 be/src/catalog/catalog-util.cc                     |   4 +-
 be/src/exec/hdfs-table-sink.cc                     |   2 +-
 be/src/exec/parquet/hdfs-parquet-table-writer.cc   | 104 +++++++++++----------
 be/src/exec/parquet/parquet-common.cc              |  21 +++--
 be/src/exec/parquet/parquet-metadata-utils.cc      |   3 +-
 be/src/experiments/compression-test.cc             |   7 +-
 be/src/service/child-query.cc                      |  16 +++-
 be/src/service/query-options-test.cc               |  55 ++++++++++-
 be/src/service/query-options.cc                    |  53 ++++++++++-
 be/src/util/codec.cc                               |  24 ++++-
 be/src/util/codec.h                                |  14 ++-
 be/src/util/compress.cc                            |  24 ++++-
 be/src/util/compress.h                             |  18 ++++
 be/src/util/decompress-test.cc                     |  39 +++++---
 be/src/util/decompress.cc                          |  25 ++++-
 be/src/util/decompress.h                           |  12 +++
 be/src/util/runtime-profile.cc                     |   4 +-
 bin/bootstrap_toolchain.py                         |   2 +-
 bin/impala-config.sh                               |   2 +
 cmake_modules/FindZstd.cmake                       |  66 +++++++++++++
 common/thrift/CatalogObjects.thrift                |   8 ++
 common/thrift/ImpalaInternalService.thrift         |   2 +-
 common/thrift/generate_error_codes.py              |   2 +
 .../QueryTest/insert_parquet_multi_codecs.test     |  42 +++++++++
 .../functional-query/queries/QueryTest/set.test    |   4 +-
 tests/common/test_dimensions.py                    |   2 +-
 tests/query_test/test_insert.py                    |   2 +-
 tests/query_test/test_insert_parquet.py            |  32 ++++++-
 30 files changed, 497 insertions(+), 98 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index ebccb69..cda8a8a 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -106,6 +106,7 @@ set_dep_root(LIBUNWIND)
 set_dep_root(LLVM)
 set(LLVM_DEBUG_ROOT $ENV{IMPALA_TOOLCHAIN}/llvm-$ENV{IMPALA_LLVM_DEBUG_VERSION})
 set_dep_root(LZ4)
+set_dep_root(ZSTD)
 set_dep_root(OPENLDAP)
 set_dep_root(PROTOBUF)
 set_dep_root(RE2)
@@ -284,6 +285,10 @@ IMPALA_ADD_THIRDPARTY_LIB(snappy ${SNAPPY_INCLUDE_DIR} ${SNAPPY_STATIC_LIB} "")
 find_package(Lz4 REQUIRED)
 IMPALA_ADD_THIRDPARTY_LIB(lz4 ${LZ4_INCLUDE_DIR} ${LZ4_STATIC_LIB} "")
 
+# find zstd lib
+find_package(Zstd REQUIRED)
+IMPALA_ADD_THIRDPARTY_LIB(zstd ${ZSTD_INCLUDE_DIR} ${ZSTD_STATIC_LIB} "")
+
 # find re2 headers and libs
 find_package(Re2 REQUIRED)
 IMPALA_ADD_THIRDPARTY_LIB(re2 ${RE2_INCLUDE_DIR} ${RE2_STATIC_LIB} "")
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 837e4af..682f341 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -480,6 +480,7 @@ endif ()
 set (IMPALA_DEPENDENCIES
   snappy
   lz4
+  zstd
   re2
   ${Boost_LIBRARIES}
   ${LLVM_MODULE_LIBS}
diff --git a/be/src/catalog/catalog-util.cc b/be/src/catalog/catalog-util.cc
index 83669c4..60c4896 100644
--- a/be/src/catalog/catalog-util.cc
+++ b/be/src/catalog/catalog-util.cc
@@ -342,8 +342,8 @@ Status TPrivilegeFromObjectName(const string& object_name, TPrivilege* privilege
 
 Status CompressCatalogObject(const uint8_t* src, uint32_t size, string* dst) {
   scoped_ptr<Codec> compressor;
-  RETURN_IF_ERROR(Codec::CreateCompressor(nullptr, false, THdfsCompression::LZ4,
-      &compressor));
+  Codec::CodecInfo codec_info(THdfsCompression::LZ4);
+  RETURN_IF_ERROR(Codec::CreateCompressor(nullptr, false, codec_info, &compressor));
   int64_t compressed_data_len = compressor->MaxOutputLen(size);
   int64_t output_buffer_len = compressed_data_len + sizeof(uint32_t);
   dst->resize(static_cast<size_t>(output_buffer_len));
diff --git a/be/src/exec/hdfs-table-sink.cc b/be/src/exec/hdfs-table-sink.cc
index dab9992..0367561 100644
--- a/be/src/exec/hdfs-table-sink.cc
+++ b/be/src/exec/hdfs-table-sink.cc
@@ -476,7 +476,7 @@ Status HdfsTableSink::InitOutputPartition(RuntimeState* state,
   }
   if (partition_descriptor.file_format() == THdfsFileFormat::TEXT &&
       state->query_options().__isset.compression_codec &&
-      state->query_options().compression_codec != THdfsCompression::NONE) {
+      state->query_options().compression_codec.codec != THdfsCompression::NONE) {
     stringstream error_msg;
     error_msg << "Writing to compressed text table is not supported. ";
     return Status(error_msg.str());
diff --git a/be/src/exec/parquet/hdfs-parquet-table-writer.cc b/be/src/exec/parquet/hdfs-parquet-table-writer.cc
index b3c1967..1a6aaf7 100644
--- a/be/src/exec/parquet/hdfs-parquet-table-writer.cc
+++ b/be/src/exec/parquet/hdfs-parquet-table-writer.cc
@@ -94,10 +94,10 @@ class HdfsParquetTableWriter::BaseColumnWriter {
  public:
   // expr - the expression to generate output values for this column.
   BaseColumnWriter(HdfsParquetTableWriter* parent, ScalarExprEvaluator* expr_eval,
-      const THdfsCompression::type& codec)
+      const Codec::CodecInfo& codec_info)
     : parent_(parent),
       expr_eval_(expr_eval),
-      codec_(codec),
+      codec_info_(codec_info),
       page_size_(DEFAULT_DATA_PAGE_SIZE),
       current_page_(nullptr),
       num_values_(0),
@@ -122,7 +122,7 @@ class HdfsParquetTableWriter::BaseColumnWriter {
   // Called after the constructor to initialize the column writer.
   Status Init() WARN_UNUSED_RESULT {
     Reset();
-    RETURN_IF_ERROR(Codec::CreateCompressor(nullptr, false, codec_, &compressor_));
+    RETURN_IF_ERROR(Codec::CreateCompressor(nullptr, false, codec_info_, &compressor_));
     return Status::OK();
   }
 
@@ -203,7 +203,7 @@ class HdfsParquetTableWriter::BaseColumnWriter {
   uint64_t total_compressed_size() const { return total_compressed_byte_size_; }
   uint64_t total_uncompressed_size() const { return total_uncompressed_byte_size_; }
   parquet::CompressionCodec::type GetParquetCodec() const {
-    return ConvertImpalaToParquetCodec(codec_);
+    return ConvertImpalaToParquetCodec(codec_info_.format_);
   }
 
  protected:
@@ -316,7 +316,7 @@ class HdfsParquetTableWriter::BaseColumnWriter {
   HdfsParquetTableWriter* parent_;
   ScalarExprEvaluator* expr_eval_;
 
-  THdfsCompression::type codec_;
+  Codec::CodecInfo codec_info_;
 
   // Compression codec for this column.  If nullptr, this column is will not be
   // compressed.
@@ -401,8 +401,8 @@ class HdfsParquetTableWriter::ColumnWriter :
     public HdfsParquetTableWriter::BaseColumnWriter {
  public:
   ColumnWriter(HdfsParquetTableWriter* parent, ScalarExprEvaluator* eval,
-      const THdfsCompression::type& codec)
-    : BaseColumnWriter(parent, eval, codec),
+      const Codec::CodecInfo& codec_info)
+    : BaseColumnWriter(parent, eval, codec_info),
       num_values_since_dict_size_check_(0),
       plain_encoded_value_size_(
           ParquetPlainEncoder::EncodedByteSize(eval->root().type())) {
@@ -521,8 +521,8 @@ class HdfsParquetTableWriter::BoolColumnWriter :
     public HdfsParquetTableWriter::BaseColumnWriter {
  public:
   BoolColumnWriter(HdfsParquetTableWriter* parent, ScalarExprEvaluator* eval,
-      const THdfsCompression::type& codec)
-    : BaseColumnWriter(parent, eval, codec),
+      const Codec::CodecInfo& codec_info)
+    : BaseColumnWriter(parent, eval, codec_info),
       page_stats_(parent_->reusable_col_mem_pool_.get(), -1),
       row_group_stats_(parent_->reusable_col_mem_pool_.get(), -1) {
     DCHECK_EQ(eval->root().type().type, TYPE_BOOLEAN);
@@ -576,11 +576,11 @@ class HdfsParquetTableWriter::BoolColumnWriter :
 class HdfsParquetTableWriter::Int64TimestampColumnWriterBase :
     public HdfsParquetTableWriter::ColumnWriter<int64_t> {
 public:
-  Int64TimestampColumnWriterBase(HdfsParquetTableWriter* parent,
-      ScalarExprEvaluator* eval, const THdfsCompression::type& codec)
-    : HdfsParquetTableWriter::ColumnWriter<int64_t>(parent, eval, codec) {
-    int64_t dummy;
-    plain_encoded_value_size_ = ParquetPlainEncoder::ByteSize(dummy);
+ Int64TimestampColumnWriterBase(HdfsParquetTableWriter* parent, ScalarExprEvaluator* eval,
+     const Codec::CodecInfo& codec_info)
+   : HdfsParquetTableWriter::ColumnWriter<int64_t>(parent, eval, codec_info) {
+   int64_t dummy;
+   plain_encoded_value_size_ = ParquetPlainEncoder::ByteSize(dummy);
   }
 
 protected:
@@ -603,9 +603,9 @@ private:
 class HdfsParquetTableWriter::Int64MilliTimestampColumnWriter :
     public HdfsParquetTableWriter::Int64TimestampColumnWriterBase {
 public:
-  Int64MilliTimestampColumnWriter(HdfsParquetTableWriter* parent,
-      ScalarExprEvaluator* eval, const THdfsCompression::type& codec)
-    : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec) {}
+ Int64MilliTimestampColumnWriter(HdfsParquetTableWriter* parent,
+     ScalarExprEvaluator* eval, const Codec::CodecInfo& codec_info)
+   : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec_info) {}
 
 protected:
   virtual bool ConvertTimestamp(const TimestampValue& ts, int64_t* result) {
@@ -617,9 +617,9 @@ protected:
 class HdfsParquetTableWriter::Int64MicroTimestampColumnWriter :
     public HdfsParquetTableWriter::Int64TimestampColumnWriterBase {
 public:
-  Int64MicroTimestampColumnWriter(HdfsParquetTableWriter* parent,
-      ScalarExprEvaluator* eval, const THdfsCompression::type& codec)
-    : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec) {}
+ Int64MicroTimestampColumnWriter(HdfsParquetTableWriter* parent,
+     ScalarExprEvaluator* eval, const Codec::CodecInfo& codec_info)
+   : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec_info) {}
 
 protected:
   virtual bool ConvertTimestamp(const TimestampValue& ts, int64_t* result) {
@@ -632,9 +632,9 @@ protected:
 class HdfsParquetTableWriter::Int64NanoTimestampColumnWriter :
     public HdfsParquetTableWriter::Int64TimestampColumnWriterBase {
 public:
-  Int64NanoTimestampColumnWriter(HdfsParquetTableWriter* parent,
-      ScalarExprEvaluator* eval, const THdfsCompression::type& codec)
-    : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec) {}
+ Int64NanoTimestampColumnWriter(HdfsParquetTableWriter* parent, ScalarExprEvaluator* eval,
+     const Codec::CodecInfo& codec_info)
+   : HdfsParquetTableWriter::Int64TimestampColumnWriterBase(parent, eval, codec_info) {}
 
 protected:
   virtual bool ConvertTimestamp(const TimestampValue& ts, int64_t* result) {
@@ -956,19 +956,25 @@ Status HdfsParquetTableWriter::Init() {
 
   // Default to snappy compressed
   THdfsCompression::type codec = THdfsCompression::SNAPPY;
-
+  // Compression level only supported for zstd.
+  int clevel = ZSTD_CLEVEL_DEFAULT;
   const TQueryOptions& query_options = state_->query_options();
   if (query_options.__isset.compression_codec) {
-    codec = query_options.compression_codec;
+    codec = query_options.compression_codec.codec;
+    clevel = query_options.compression_codec.compression_level;
   }
   if (!(codec == THdfsCompression::NONE ||
         codec == THdfsCompression::GZIP ||
-        codec == THdfsCompression::SNAPPY)) {
+        codec == THdfsCompression::SNAPPY ||
+        codec == THdfsCompression::ZSTD)) {
     stringstream ss;
     ss << "Invalid parquet compression codec " << Codec::GetCodecName(codec);
     return Status(ss.str());
   }
   VLOG_FILE << "Using compression codec: " << codec;
+  if (codec == THdfsCompression::ZSTD) {
+    VLOG_FILE << "Using compression level: " << clevel;
+  }
 
   if (query_options.__isset.parquet_page_row_count_limit) {
     page_row_count_limit_ = query_options.parquet_page_row_count_limit;
@@ -986,6 +992,8 @@ Status HdfsParquetTableWriter::Init() {
         PrettyPrinter::Print(min_block_size, TUnit::BYTES), num_cols));
   }
 
+  Codec::CodecInfo codec_info(codec, clevel);
+
   columns_.resize(num_cols);
   // Initialize each column structure.
   for (int i = 0; i < columns_.size(); ++i) {
@@ -993,43 +1001,43 @@ Status HdfsParquetTableWriter::Init() {
     const ColumnType& type = output_expr_evals_[i]->root().type();
     switch (type.type) {
       case TYPE_BOOLEAN:
-        writer = new BoolColumnWriter(this, output_expr_evals_[i], codec);
+        writer = new BoolColumnWriter(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_TINYINT:
-        writer = new ColumnWriter<int8_t>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<int8_t>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_SMALLINT:
-        writer = new ColumnWriter<int16_t>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<int16_t>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_INT:
-        writer = new ColumnWriter<int32_t>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<int32_t>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_BIGINT:
-        writer = new ColumnWriter<int64_t>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<int64_t>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_FLOAT:
-        writer = new ColumnWriter<float>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<float>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_DOUBLE:
-        writer = new ColumnWriter<double>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<double>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_TIMESTAMP:
         switch (state_->query_options().parquet_timestamp_type) {
           case TParquetTimestampType::INT96_NANOS:
             writer =
-                new ColumnWriter<TimestampValue>(this, output_expr_evals_[i], codec);
+                new ColumnWriter<TimestampValue>(this, output_expr_evals_[i], codec_info);
             break;
           case TParquetTimestampType::INT64_MILLIS:
-            writer =
-                new Int64MilliTimestampColumnWriter(this, output_expr_evals_[i], codec);
+            writer = new Int64MilliTimestampColumnWriter(
+                this, output_expr_evals_[i], codec_info);
             break;
           case TParquetTimestampType::INT64_MICROS:
-            writer =
-                new Int64MicroTimestampColumnWriter(this, output_expr_evals_[i], codec);
+            writer = new Int64MicroTimestampColumnWriter(
+                this, output_expr_evals_[i], codec_info);
             break;
           case TParquetTimestampType::INT64_NANOS:
-            writer =
-                new Int64NanoTimestampColumnWriter(this, output_expr_evals_[i], codec);
+            writer = new Int64NanoTimestampColumnWriter(
+                this, output_expr_evals_[i], codec_info);
             break;
           default:
             DCHECK(false);
@@ -1038,28 +1046,28 @@ Status HdfsParquetTableWriter::Init() {
       case TYPE_VARCHAR:
       case TYPE_STRING:
       case TYPE_CHAR:
-        writer = new ColumnWriter<StringValue>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<StringValue>(this, output_expr_evals_[i], codec_info);
         break;
       case TYPE_DECIMAL:
         switch (output_expr_evals_[i]->root().type().GetByteSize()) {
           case 4:
-            writer = new ColumnWriter<Decimal4Value>(
-                this, output_expr_evals_[i], codec);
+            writer =
+                new ColumnWriter<Decimal4Value>(this, output_expr_evals_[i], codec_info);
             break;
           case 8:
-            writer = new ColumnWriter<Decimal8Value>(
-                this, output_expr_evals_[i], codec);
+            writer =
+                new ColumnWriter<Decimal8Value>(this, output_expr_evals_[i], codec_info);
             break;
           case 16:
-            writer = new ColumnWriter<Decimal16Value>(
-                this, output_expr_evals_[i], codec);
+            writer =
+                new ColumnWriter<Decimal16Value>(this, output_expr_evals_[i], codec_info);
             break;
           default:
             DCHECK(false);
         }
         break;
       case TYPE_DATE:
-        writer = new ColumnWriter<DateValue>(this, output_expr_evals_[i], codec);
+        writer = new ColumnWriter<DateValue>(this, output_expr_evals_[i], codec_info);
         break;
       default:
         DCHECK(false);
diff --git a/be/src/exec/parquet/parquet-common.cc b/be/src/exec/parquet/parquet-common.cc
index e7ee624..20b23dc 100644
--- a/be/src/exec/parquet/parquet-common.cc
+++ b/be/src/exec/parquet/parquet-common.cc
@@ -24,8 +24,10 @@ const THdfsCompression::type PARQUET_TO_IMPALA_CODEC[] = {
   THdfsCompression::NONE,
   THdfsCompression::SNAPPY,
   THdfsCompression::GZIP,
-  THdfsCompression::LZO
-};
+  THdfsCompression::LZO,
+  THdfsCompression::BROTLI,
+  THdfsCompression::LZ4,
+  THdfsCompression::ZSTD};
 
 const int PARQUET_TO_IMPALA_CODEC_SIZE =
     sizeof(PARQUET_TO_IMPALA_CODEC) / sizeof(PARQUET_TO_IMPALA_CODEC[0]);
@@ -33,13 +35,18 @@ const int PARQUET_TO_IMPALA_CODEC_SIZE =
 /// Mapping of Impala codec enums to Parquet enums
 const parquet::CompressionCodec::type IMPALA_TO_PARQUET_CODEC[] = {
   parquet::CompressionCodec::UNCOMPRESSED,
-  parquet::CompressionCodec::SNAPPY,  // DEFAULT
-  parquet::CompressionCodec::GZIP,    // GZIP
-  parquet::CompressionCodec::GZIP,    // DEFLATE
+  parquet::CompressionCodec::SNAPPY, // DEFAULT
+  parquet::CompressionCodec::GZIP,   // GZIP
+  parquet::CompressionCodec::GZIP,   // DEFLATE
+  // Placeholder for BZIP2 which isn't a valid parquet codec.
+  parquet::CompressionCodec::SNAPPY, // BZIP2
   parquet::CompressionCodec::SNAPPY,
-  parquet::CompressionCodec::SNAPPY,  // SNAPPY_BLOCKED
+  parquet::CompressionCodec::SNAPPY, // SNAPPY_BLOCKED
   parquet::CompressionCodec::LZO,
-};
+  parquet::CompressionCodec::LZ4,
+  parquet::CompressionCodec::GZIP,   // ZLIB
+  parquet::CompressionCodec::ZSTD,
+  parquet::CompressionCodec::BROTLI};
 
 const int IMPALA_TO_PARQUET_CODEC_SIZE =
     sizeof(IMPALA_TO_PARQUET_CODEC) / sizeof(IMPALA_TO_PARQUET_CODEC[0]);
diff --git a/be/src/exec/parquet/parquet-metadata-utils.cc b/be/src/exec/parquet/parquet-metadata-utils.cc
index 6d9d4de..02544ca 100644
--- a/be/src/exec/parquet/parquet-metadata-utils.cc
+++ b/be/src/exec/parquet/parquet-metadata-utils.cc
@@ -283,7 +283,8 @@ Status ParquetMetadataUtils::ValidateRowGroupColumn(
   const auto codec = Ubsan::EnumToInt(&col_chunk_metadata.codec);
   if (codec != parquet::CompressionCodec::UNCOMPRESSED &&
       codec != parquet::CompressionCodec::SNAPPY &&
-      codec != parquet::CompressionCodec::GZIP) {
+      codec != parquet::CompressionCodec::GZIP &&
+      codec != parquet::CompressionCodec::ZSTD) {
     return Status(Substitute("File '$0' uses an unsupported compression: $1 for column "
         "'$2'.", filename, codec, schema_element.name));
   }
diff --git a/be/src/experiments/compression-test.cc b/be/src/experiments/compression-test.cc
index 48ed666..25d392a 100644
--- a/be/src/experiments/compression-test.cc
+++ b/be/src/experiments/compression-test.cc
@@ -44,7 +44,7 @@ namespace impala {
 
 // Generates num strings between min_len and max_len.
 // Outputs the uncompressed/compressed/sorted_compressed sizes.
-void TestCompression(int num, int min_len, int max_len, THdfsCompression::type codec) {
+void TestCompression(int num, int min_len, int max_len, THdfsCompression::type format) {
   vector<string> strings;
   uint8_t* buffer = (uint8_t*)malloc(max_len * num);
   int offset = 0;
@@ -69,7 +69,8 @@ void TestCompression(int num, int min_len, int max_len, THdfsCompression::type c
   }
 
   scoped_ptr<Codec> compressor;
-  Status status = Codec::CreateCompressor(NULL, false, codec, &compressor);
+  Codec::CodecInfo codec_info(format);
+  Status status = Codec::CreateCompressor(NULL, false, codec_info, &compressor);
   DCHECK(status.ok());
 
   int64_t compressed_len = compressor->MaxOutputLen(offset);
@@ -83,7 +84,7 @@ void TestCompression(int num, int min_len, int max_len, THdfsCompression::type c
         &sorted_compressed_len, &sorted_compressed_buffer));
 
   cout << "NumStrings=" << num << " MinLen=" << min_len << " MaxLen=" << max_len
-       << " Codec=" << codec << endl;
+       << " Codec=" << codec_info.format_ << endl;
   cout << "  Uncompressed len: " << offset << endl;
   cout << "  Compressed len: " << compressed_len << endl;
   cout << "  Sorted Compressed len: " << sorted_compressed_len << endl;
diff --git a/be/src/service/child-query.cc b/be/src/service/child-query.cc
index 528ef81..d690d4e 100644
--- a/be/src/service/child-query.cc
+++ b/be/src/service/child-query.cc
@@ -125,13 +125,27 @@ Status ChildQuery::ExecAndFetch() {
   return status;
 }
 
+template <typename T>
+void PrintQueryOptionValue (const T& option, stringstream& val) {
+  val << option;
+}
+
+void PrintQueryOptionValue(const impala::TCompressionCodec& compression_codec,
+    stringstream& val) {
+  if (compression_codec.codec != THdfsCompression::ZSTD) {
+    val << compression_codec.codec;
+  } else {
+    val << compression_codec.codec << ":" << compression_codec.compression_level;
+  }
+}
+
 void ChildQuery::SetQueryOptions(const TQueryOptions& parent_options,
     TExecuteStatementReq* exec_stmt_req) {
   map<string, string> conf;
 #define QUERY_OPT_FN(NAME, ENUM, LEVEL)\
   if (parent_options.__isset.NAME) {\
     stringstream val;\
-    val << parent_options.NAME;\
+    PrintQueryOptionValue(parent_options.NAME, val);\
     conf[#ENUM] = val.str();\
   }
 #define REMOVED_QUERY_OPT_FN(NAME, ENUM)
diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 2b149d3..97b0b71 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -17,9 +17,11 @@
 
 #include "service/query-options.h"
 
+#include <zstd.h>
 #include <boost/preprocessor/seq/for_each.hpp>
 #include <boost/preprocessor/tuple/to_seq.hpp>
 
+#include "gutil/strings/substitute.h"
 #include "runtime/runtime-filter.h"
 #include "testutil/gtest-util.h"
 #include "util/mem-info.h"
@@ -27,6 +29,7 @@
 using namespace boost;
 using namespace impala;
 using namespace std;
+using namespace strings;
 
 constexpr int32_t I32_MAX = numeric_limits<int32_t>::max();
 constexpr int64_t I64_MAX = numeric_limits<int64_t>::max();
@@ -209,9 +212,6 @@ TEST(QueryOptions, SetEnumOptions) {
       TParquetFallbackSchemaResolution, (POSITION, NAME)), true);
   TestEnumCase(options, CASE(parquet_array_resolution, TParquetArrayResolution,
       (THREE_LEVEL, TWO_LEVEL, TWO_LEVEL_THEN_THREE_LEVEL)), true);
-  TestEnumCase(options, CASE(compression_codec, THdfsCompression,
-      (NONE, DEFAULT, GZIP, DEFLATE, BZIP2, SNAPPY, SNAPPY_BLOCKED, LZO, LZ4, ZLIB,
-          ZSTD)), true);
   TestEnumCase(options, CASE(default_file_format, THdfsFileFormat,
       (TEXT, RC_FILE, SEQUENCE_FILE, AVRO, PARQUET, KUDU, ORC)), true);
   TestEnumCase(options, CASE(runtime_filter_mode, TRuntimeFilterMode,
@@ -227,7 +227,7 @@ TEST(QueryOptions, SetEnumOptions) {
 TEST(QueryOptions, SetIntOptions) {
   TQueryOptions options;
   // List of pairs of Key and its valid range
-  pair<OptionDef<int32_t>, Range<int32_t>> case_set[] {
+  pair<OptionDef<int32_t>, Range<int32_t>> case_set[]{
       {MAKE_OPTIONDEF(runtime_filter_wait_time_ms),    {0, I32_MAX}},
       {MAKE_OPTIONDEF(mt_dop),                         {0, 64}},
       {MAKE_OPTIONDEF(disable_codegen_rows_threshold), {0, I32_MAX}},
@@ -462,4 +462,51 @@ TEST(QueryOptions, ResetToDefaultViaEmptyString) {
   }
 }
 
+TEST(QueryOptions, CompressionCodec) {
+#define ENTRY(_, prefix, entry) (prefix::entry),
+#define ENTRIES(prefix, name) BOOST_PP_SEQ_FOR_EACH(ENTRY, prefix, name)
+#define CASE(enumtype, enums) {ENTRIES(enumtype, BOOST_PP_TUPLE_TO_SEQ(enums))}
+  TQueryOptions options;
+  vector<THdfsCompression::type> codecs = CASE(THdfsCompression, (NONE, DEFAULT, GZIP,
+      DEFLATE, BZIP2, SNAPPY, SNAPPY_BLOCKED, LZO, LZ4, ZLIB, ZSTD, BROTLI));
+  // Test valid values for compression_codec.
+  for (auto& codec : codecs) {
+    EXPECT_TRUE(SetQueryOption("compression_codec", Substitute("$0",codec), &options,
+        nullptr).ok());
+    // Test that compression level is only supported for ZSTD.
+    if (codec != THdfsCompression::ZSTD) {
+      EXPECT_FALSE(SetQueryOption("compression_codec", Substitute("$0:1",codec),
+        &options, nullptr).ok());
+    }
+    else {
+      EXPECT_TRUE(SetQueryOption("compression_codec",
+          Substitute("zstd:$0",ZSTD_CLEVEL_DEFAULT), &options, nullptr).ok());
+    }
+  }
+
+  // Test invalid values for compression_codec.
+  EXPECT_FALSE(SetQueryOption("compression_codec", Substitute("$0", codecs.back() + 1),
+      &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec", "foo", &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec", "1%", &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec", "-1", &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec", ":", &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec", ":1", &options, nullptr).ok());
+
+  // Test compression levels for ZSTD.
+  const int zstd_min_clevel = 1;
+  const int zstd_max_clevel = ZSTD_maxCLevel();
+  for (int i = zstd_min_clevel; i <= zstd_max_clevel; i++)
+  {
+    EXPECT_TRUE(SetQueryOption("compression_codec", Substitute("ZSTD:$0",i), &options,
+      nullptr).ok());
+  }
+  EXPECT_FALSE(SetQueryOption("compression_codec",
+    Substitute("ZSTD:$0", zstd_min_clevel - 1), &options, nullptr).ok());
+  EXPECT_FALSE(SetQueryOption("compression_codec",
+    Substitute("ZSTD:$0", zstd_max_clevel + 1), &options, nullptr).ok());
+#undef CASE
+#undef ENTRIES
+#undef ENTRY
+}
 IMPALA_TEST_MAIN();
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index e10cb3e..dc1c365 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -25,10 +25,11 @@
 #include "exprs/timezone_db.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 
+#include <zstd.h>
 #include <sstream>
 #include <boost/algorithm/string.hpp>
-#include <gutil/strings/substitute.h>
 #include <gutil/strings/strip.h>
+#include <gutil/strings/substitute.h>
 
 #include "common/names.h"
 
@@ -89,6 +90,15 @@ const string& PrintQueryOptionValue(const std::string& option)  {
   return option;
 }
 
+const string PrintQueryOptionValue(const impala::TCompressionCodec& compression_codec) {
+  if (compression_codec.codec != THdfsCompression::ZSTD) {
+    return Substitute("$0", PrintThriftEnum(compression_codec.codec));
+  } else {
+    return Substitute("$0:$1", PrintThriftEnum(compression_codec.codec),
+        compression_codec.compression_level);
+  }
+}
+
 void impala::TQueryOptionsToMap(const TQueryOptions& query_options,
     map<string, string>* configuration) {
 #define QUERY_OPT_FN(NAME, ENUM, LEVEL)\
@@ -128,6 +138,11 @@ static TQueryOptions DefaultQueryOptions() {
   return defaults;
 }
 
+inline bool operator!=(const TCompressionCodec& a,
+  const TCompressionCodec& b) {
+  return (a.codec != b.codec || a.compression_level != b.compression_level);
+}
+
 string impala::DebugQueryOptions(const TQueryOptions& query_options) {
   const static TQueryOptions defaults = DefaultQueryOptions();
   int i = 0;
@@ -264,10 +279,42 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_debug_action(value.c_str());
         break;
       case TImpalaQueryOptions::COMPRESSION_CODEC: {
+        // Acceptable values are:
+        // - zstd:compression_level
+        // - codec
+        vector<string> tokens;
+        split(tokens, value, is_any_of(":"), token_compress_on);
+        if (tokens.size() > 2) return Status("Invalid compression codec value");
+
+        string& codec_name = tokens[0];
+        trim(codec_name);
+        int compression_level = ZSTD_CLEVEL_DEFAULT;
         THdfsCompression::type enum_type;
-        RETURN_IF_ERROR(GetThriftEnum(value, "compression codec",
+        RETURN_IF_ERROR(GetThriftEnum(codec_name, "compression codec",
             _THdfsCompression_VALUES_TO_NAMES, &enum_type));
-        query_options->__set_compression_codec(enum_type);
+
+        if (tokens.size() == 2) {
+          if (enum_type != THdfsCompression::ZSTD) {
+            return Status("Compression level only supported for ZSTD");
+          }
+          StringParser::ParseResult status;
+          string& clevel = tokens[1];
+          trim(clevel);
+          compression_level = StringParser::StringToInt<int>(
+            clevel.c_str(), static_cast<int>(clevel.size()), &status);
+          if (status != StringParser::PARSE_SUCCESS || compression_level < 1
+              || compression_level > ZSTD_maxCLevel()) {
+            return Status(Substitute("Invalid ZSTD compression level '$0'."
+                " Valid values are in [1,$1]", clevel, ZSTD_maxCLevel()));
+          }
+        }
+
+        TCompressionCodec compression_codec;
+        compression_codec.__set_codec(enum_type);
+        if (enum_type == THdfsCompression::ZSTD) {
+          compression_codec.__set_compression_level(compression_level);
+        }
+        query_options->__set_compression_codec(compression_codec);
         break;
       }
       case TImpalaQueryOptions::HBASE_CACHING:
diff --git a/be/src/util/codec.cc b/be/src/util/codec.cc
index c76fa44..8f19eee 100644
--- a/be/src/util/codec.cc
+++ b/be/src/util/codec.cc
@@ -34,6 +34,9 @@ const char* const Codec::DEFAULT_COMPRESSION =
 const char* const Codec::GZIP_COMPRESSION = "org.apache.hadoop.io.compress.GzipCodec";
 const char* const Codec::BZIP2_COMPRESSION = "org.apache.hadoop.io.compress.BZip2Codec";
 const char* const Codec::SNAPPY_COMPRESSION = "org.apache.hadoop.io.compress.SnappyCodec";
+const char* const Codec::LZ4_COMPRESSION = "org.apache.hadoop.io.compress.Lz4Codec";
+const char* const Codec::ZSTD_COMPRESSION =
+    "org.apache.hadoop.io.compress.ZStandardCodec";
 const char* const Codec::UNKNOWN_CODEC_ERROR =
     "This compression codec is currently unsupported: ";
 const char* const NO_LZO_MSG = "LZO codecs may not be created via the Codec interface. "
@@ -43,7 +46,9 @@ const Codec::CodecMap Codec::CODEC_MAP = {{"", THdfsCompression::NONE},
     {DEFAULT_COMPRESSION, THdfsCompression::DEFAULT},
     {GZIP_COMPRESSION, THdfsCompression::GZIP},
     {BZIP2_COMPRESSION, THdfsCompression::BZIP2},
-    {SNAPPY_COMPRESSION, THdfsCompression::SNAPPY_BLOCKED}};
+    {SNAPPY_COMPRESSION, THdfsCompression::SNAPPY_BLOCKED},
+    {LZ4_COMPRESSION, THdfsCompression::LZ4},
+    {ZSTD_COMPRESSION, THdfsCompression::ZSTD}};
 
 string Codec::GetCodecName(THdfsCompression::type type) {
   for (const CodecMap::value_type& codec: g_CatalogObjects_constants.COMPRESSION_MAP) {
@@ -71,13 +76,15 @@ Status Codec::CreateCompressor(MemPool* mem_pool, bool reuse, const string& code
     return Status(Substitute("$0$1", UNKNOWN_CODEC_ERROR, codec));
   }
 
-  RETURN_IF_ERROR(
-      CreateCompressor(mem_pool, reuse, type->second, compressor));
+  CodecInfo codec_info(
+      type->second, (type->second == THdfsCompression::ZSTD) ? ZSTD_CLEVEL_DEFAULT : 0);
+  RETURN_IF_ERROR(CreateCompressor(mem_pool, reuse, codec_info, compressor));
   return Status::OK();
 }
 
-Status Codec::CreateCompressor(MemPool* mem_pool, bool reuse,
-    THdfsCompression::type format, scoped_ptr<Codec>* compressor) {
+Status Codec::CreateCompressor(MemPool* mem_pool, bool reuse, const CodecInfo& codec_info,
+    scoped_ptr<Codec>* compressor) {
+  THdfsCompression::type format = codec_info.format_;
   switch (format) {
     case THdfsCompression::NONE:
       compressor->reset(nullptr);
@@ -103,6 +110,10 @@ Status Codec::CreateCompressor(MemPool* mem_pool, bool reuse,
     case THdfsCompression::LZ4:
       compressor->reset(new Lz4Compressor(mem_pool, reuse));
       break;
+    case THdfsCompression::ZSTD:
+      compressor->reset(new ZstandardCompressor(mem_pool, reuse,
+          codec_info.compression_level_));
+      break;
     default: {
       if (format == THdfsCompression::LZO) return Status(NO_LZO_MSG);
       return Status(Substitute("Unsupported codec: $0", format));
@@ -149,6 +160,9 @@ Status Codec::CreateDecompressor(MemPool* mem_pool, bool reuse,
     case THdfsCompression::LZ4:
       decompressor->reset(new Lz4Decompressor(mem_pool, reuse));
       break;
+    case THdfsCompression::ZSTD:
+      decompressor->reset(new ZstandardDecompressor(mem_pool, reuse));
+      break;
     default: {
       if (format == THdfsCompression::LZO) return Status(NO_LZO_MSG);
       return Status(Substitute("Unsupported codec: $0", format));
diff --git a/be/src/util/codec.h b/be/src/util/codec.h
index 8d6b88f..c17bc7d 100644
--- a/be/src/util/codec.h
+++ b/be/src/util/codec.h
@@ -44,6 +44,8 @@ class Codec {
   static const char* const GZIP_COMPRESSION;
   static const char* const BZIP2_COMPRESSION;
   static const char* const SNAPPY_COMPRESSION;
+  static const char* const LZ4_COMPRESSION;
+  static const char* const ZSTD_COMPRESSION;
   static const char* const UNKNOWN_CODEC_ERROR;
 
   // Output buffer size for streaming compressed file.
@@ -53,6 +55,16 @@ class Codec {
   typedef std::map<const std::string, const THdfsCompression::type> CodecMap;
   static const CodecMap CODEC_MAP;
 
+  struct CodecInfo {
+   public:
+    CodecInfo(THdfsCompression::type format, int compression_level = 0)
+      : format_(format), compression_level_(compression_level) {}
+
+    THdfsCompression::type format_;
+    // Currently only ZSTD uses compression level.
+    int compression_level_;
+  };
+
   /// Create a decompressor.
   /// Input:
   ///  mem_pool: the memory pool used to store the decompressed data.
@@ -79,7 +91,7 @@ class Codec {
   /// Output:
   ///  compressor: scoped pointer to the compressor class to use.
   static Status CreateCompressor(MemPool* mem_pool, bool reuse,
-      THdfsCompression::type format,
+      const CodecInfo& codec_info,
       boost::scoped_ptr<Codec>* compressor) WARN_UNUSED_RESULT;
 
   /// Alternate factory method: takes a codec string and populates a scoped pointer.
diff --git a/be/src/util/compress.cc b/be/src/util/compress.cc
index 7a0181b..847168d 100644
--- a/be/src/util/compress.cc
+++ b/be/src/util/compress.cc
@@ -22,8 +22,10 @@
 #include <boost/crc.hpp>
 #include <gutil/strings/substitute.h>
 #undef DISALLOW_COPY_AND_ASSIGN // Snappy redefines this.
-#include <snappy.h>
 #include <lz4.h>
+#include <snappy.h>
+#include <zstd.h>
+#include <zstd_errors.h>
 
 #include "exec/read-write-util.h"
 #include "runtime/mem-pool.h"
@@ -318,3 +320,23 @@ Status Lz4Compressor::ProcessBlock(bool output_preallocated, int64_t input_lengt
       reinterpret_cast<char*>(*output), input_length, *output_length);
   return Status::OK();
 }
+
+ZstandardCompressor::ZstandardCompressor(MemPool* mem_pool, bool reuse_buffer, int clevel)
+  : Codec(mem_pool, reuse_buffer), clevel_(clevel) {}
+
+int64_t ZstandardCompressor::MaxOutputLen(int64_t input_len, const uint8_t* input) {
+  return ZSTD_compressBound(input_len);
+}
+
+Status ZstandardCompressor::ProcessBlock(bool output_preallocated, int64_t input_length,
+    const uint8_t* input, int64_t* output_length, uint8_t** output) {
+  DCHECK_GE(input_length, 0);
+  DCHECK(output_preallocated) << "Output was not allocated for Zstd Codec";
+  if (input_length == 0) return Status::OK();
+  *output_length = ZSTD_compress(*output, *output_length, input, input_length, clevel_);
+  if (ZSTD_isError(*output_length)) {
+    return Status(TErrorCode::ZSTD_ERROR, "ZSTD_compress",
+        ZSTD_getErrorString(ZSTD_getErrorCode(*output_length)));
+  }
+  return Status::OK();
+}
diff --git a/be/src/util/compress.h b/be/src/util/compress.h
index 15d9b92..493051d 100644
--- a/be/src/util/compress.h
+++ b/be/src/util/compress.h
@@ -21,6 +21,7 @@
 
 /// We need zlib.h here to declare stream_ below.
 #include <zlib.h>
+#include <zstd.h>
 
 #include "util/codec.h"
 
@@ -131,5 +132,22 @@ class Lz4Compressor : public Codec {
   virtual std::string file_extension() const override { return "lz4"; }
 };
 
+/// ZStandard compression codec.
+class ZstandardCompressor : public Codec {
+ public:
+  ZstandardCompressor(MemPool* mem_pool = nullptr, bool reuse_buffer = false,
+      int clevel = ZSTD_CLEVEL_DEFAULT);
+  virtual ~ZstandardCompressor() {}
+
+  virtual int64_t MaxOutputLen(
+      int64_t input_len, const uint8_t* input = nullptr) override;
+  virtual Status ProcessBlock(bool output_preallocated, int64_t input_length,
+      const uint8_t* input, int64_t* output_length,
+      uint8_t** output) override WARN_UNUSED_RESULT;
+  virtual std::string file_extension() const override { return "zstd"; }
+
+ private:
+  int clevel_;
+};
 }
 #endif
diff --git a/be/src/util/decompress-test.cc b/be/src/util/decompress-test.cc
index 5284647..5d7e0ca 100644
--- a/be/src/util/decompress-test.cc
+++ b/be/src/util/decompress-test.cc
@@ -15,8 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdlib.h>
 #include <stdio.h>
+#include <stdlib.h>
+#include <zstd.h>
 #include <iostream>
 
 #include "gen-cpp/Descriptors_types.h"
@@ -24,12 +25,16 @@
 #include "runtime/mem-tracker.h"
 #include "runtime/mem-pool.h"
 #include "testutil/gtest-util.h"
+#include "testutil/rand-util.h"
 #include "util/decompress.h"
 #include "util/compress.h"
 #include "util/ubsan.h"
 
 #include "common/names.h"
 
+using std::mt19937;
+using std::uniform_int_distribution;
+
 namespace impala {
 
 // Fixture for testing class Decompressor
@@ -58,15 +63,16 @@ class DecompressorTest : public ::testing::Test {
     mem_pool_.FreeAll();
   }
 
-  void RunTest(THdfsCompression::type format) {
+  void RunTest(THdfsCompression::type format, int clevel = 0) {
     scoped_ptr<Codec> compressor;
     scoped_ptr<Codec> decompressor;
 
-    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, format, &compressor));
+    Codec::CodecInfo codec_info(format, clevel);
+    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, codec_info, &compressor));
     EXPECT_OK(Codec::CreateDecompressor(&mem_pool_, true, format, &decompressor));
 
-    // LZ4 is not implemented to work without an allocated output
-    if(format == THdfsCompression::LZ4) {
+    // LZ4 & ZSTD are not implemented to work without an allocated output
+    if (format == THdfsCompression::LZ4 || format == THdfsCompression::ZSTD) {
       CompressAndDecompressNoOutputAllocated(compressor.get(), decompressor.get(),
           sizeof(input_), input_);
       CompressAndDecompressNoOutputAllocated(compressor.get(), decompressor.get(),
@@ -97,7 +103,9 @@ class DecompressorTest : public ::testing::Test {
   void RunTestStreaming(THdfsCompression::type format) {
     scoped_ptr<Codec> compressor;
     scoped_ptr<Codec> decompressor;
-    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, format, &compressor));
+    Codec::CodecInfo codec_info(format);
+
+    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, codec_info, &compressor));
     EXPECT_OK(Codec::CreateDecompressor(&mem_pool_, true, format, &decompressor));
 
     CompressAndStreamingDecompress(compressor.get(), decompressor.get(),
@@ -331,7 +339,8 @@ class DecompressorTest : public ::testing::Test {
     *compressed_len = 0;
 
     scoped_ptr<Codec> compressor;
-    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, format, &compressor));
+    Codec::CodecInfo codec_info(format);
+    EXPECT_OK(Codec::CreateCompressor(&mem_pool_, true, codec_info, &compressor));
 
     // Make sure we don't completely fill the buffer, leave at least RAW_INPUT_SIZE
     // bytes free in compressed buffer for junk data testing (Test case 3).
@@ -412,8 +421,8 @@ TEST_F(DecompressorTest, Impala1506) {
   MemTracker trax;
   MemPool pool(&trax);
   scoped_ptr<Codec> compressor;
-  EXPECT_OK(
-      Codec::CreateCompressor(&pool, true, impala::THdfsCompression::GZIP, &compressor));
+  Codec::CodecInfo codec_info(impala::THdfsCompression::GZIP);
+  EXPECT_OK(Codec::CreateCompressor(&pool, true, codec_info, &compressor));
 
   int64_t input_len = 3;
   const uint8_t input[3] = {1, 2, 3};
@@ -457,8 +466,8 @@ TEST_F(DecompressorTest, LZ4Huge) {
   for (int i = 0 ; i < payload_len; ++i) payload[i] = rand();
 
   scoped_ptr<Codec> compressor;
-  EXPECT_OK(Codec::CreateCompressor(nullptr, true, impala::THdfsCompression::LZ4,
-      &compressor));
+  Codec::CodecInfo codec_info(impala::THdfsCompression::LZ4);
+  EXPECT_OK(Codec::CreateCompressor(nullptr, true, codec_info, &compressor));
 
   // The returned max_size is 0 because the payload is too big.
   int64_t max_size = compressor->MaxOutputLen(payload_len);
@@ -472,6 +481,14 @@ TEST_F(DecompressorTest, LZ4Huge) {
       &compressed_len, &compressed_ptr), TErrorCode::LZ4_COMPRESSION_INPUT_TOO_LARGE);
 }
 
+TEST_F(DecompressorTest, ZSTD) {
+  RunTest(THdfsCompression::ZSTD, ZSTD_CLEVEL_DEFAULT);
+  mt19937 rng;
+  RandTestUtil::SeedRng("ZSTD_COMPRESSION_LEVEL_SEED", &rng);
+  // zstd supports compression levels from 1 up to ZSTD_maxCLevel()
+  const int clevel = uniform_int_distribution<int>(1, ZSTD_maxCLevel())(rng);
+  RunTest(THdfsCompression::ZSTD, clevel);
+}
 }
 
 int main(int argc, char **argv) {
diff --git a/be/src/util/decompress.cc b/be/src/util/decompress.cc
index 267e8f8..438bfbe 100644
--- a/be/src/util/decompress.cc
+++ b/be/src/util/decompress.cc
@@ -21,8 +21,10 @@
 #include <zlib.h>
 #include <bzlib.h>
 #undef DISALLOW_COPY_AND_ASSIGN // Snappy redefines this.
-#include <snappy.h>
 #include <lz4.h>
+#include <snappy.h>
+#include <zstd.h>
+#include <zstd_errors.h>
 
 #include "common/logging.h"
 #include "exec/read-write-util.h"
@@ -603,3 +605,24 @@ Status Lz4Decompressor::ProcessBlock(bool output_preallocated, int64_t input_len
   *output_length = ret;
   return Status::OK();
 }
+
+ZstandardDecompressor::ZstandardDecompressor(MemPool* mem_pool, bool reuse_buffer)
+  : Codec(mem_pool, reuse_buffer) {}
+
+int64_t ZstandardDecompressor::MaxOutputLen(int64_t input_len, const uint8_t* input) {
+  return -1;
+}
+
+Status ZstandardDecompressor::ProcessBlock(bool output_preallocated, int64_t input_length,
+    const uint8_t* input, int64_t* output_length, uint8_t** output) {
+  DCHECK(output_preallocated) << "Output was not allocated for Zstd Codec";
+  if (*output_length == 0) return Status::OK();
+  size_t ret = ZSTD_decompress(*output, *output_length, input, input_length);
+  if (ZSTD_isError(ret)) {
+    *output_length = 0;
+    return Status(TErrorCode::ZSTD_ERROR, "ZSTD_decompress",
+        ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
+  }
+  *output_length = ret;
+  return Status::OK();
+}
diff --git a/be/src/util/decompress.h b/be/src/util/decompress.h
index 61c5994..9ff66fe 100644
--- a/be/src/util/decompress.h
+++ b/be/src/util/decompress.h
@@ -132,5 +132,17 @@ class SnappyBlockDecompressor : public Codec {
   virtual std::string file_extension() const override { return "snappy"; }
 };
 
+class ZstandardDecompressor : public Codec {
+ public:
+  ZstandardDecompressor(MemPool* mem_pool, bool reuse_buffer);
+  virtual ~ZstandardDecompressor() {}
+
+  virtual int64_t MaxOutputLen(
+      int64_t input_len, const uint8_t* input = nullptr) override;
+  virtual Status ProcessBlock(bool output_preallocated, int64_t input_length,
+      const uint8_t* input, int64_t* output_length,
+      uint8_t** output) override WARN_UNUSED_RESULT;
+  virtual std::string file_extension() const override { return "zstd"; }
+};
 }
 #endif
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index 8f91b81..78ae8d1 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -860,8 +860,8 @@ Status RuntimeProfile::SerializeToArchiveString(stringstream* out) const {
   // Compress the serialized thrift string.  This uses string keys and is very
   // easy to compress.
   scoped_ptr<Codec> compressor;
-  RETURN_IF_ERROR(
-      Codec::CreateCompressor(NULL, false, THdfsCompression::DEFAULT, &compressor));
+  Codec::CodecInfo codec_info(THdfsCompression::DEFAULT);
+  RETURN_IF_ERROR(Codec::CreateCompressor(NULL, false, codec_info, &compressor));
   const auto close_compressor =
       MakeScopeExitTrigger([&compressor]() { compressor->Close(); });
 
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index 1f9b28c..ae0b7d3 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -541,7 +541,7 @@ if __name__ == "__main__":
       "avro", "binutils", "boost", "breakpad", "bzip2", "cctz", "cmake", "crcutil",
       "flatbuffers", "gcc", "gdb", "gflags", "glog", "gperftools", "gtest", "libev",
       "libunwind", "lz4", "openldap", "openssl", "orc", "protobuf",
-      "rapidjson", "re2", "snappy", "thrift", "tpc-h", "tpc-ds", "zlib"])
+      "rapidjson", "re2", "snappy", "thrift", "tpc-h", "tpc-ds", "zlib", "zstd"])
   packages.insert(0, Package("llvm", "5.0.1-asserts-p1"))
   packages.insert(0, Package("thrift", os.environ.get("IMPALA_THRIFT11_VERSION")))
   bootstrap(toolchain_root, packages)
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 316cff3..7851ff2 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -118,6 +118,8 @@ export IMPALA_LLVM_DEBUG_VERSION=5.0.1-asserts-p1
 unset IMPALA_LLVM_DEBUG_URL
 export IMPALA_LZ4_VERSION=1.7.5
 unset IMPALA_LZ4_URL
+export IMPALA_ZSTD_VERSION=1.4.0
+unset IMPALA_ZSTD_URL
 export IMPALA_OPENLDAP_VERSION=2.4.47
 unset IMPALA_OPENLDAP_URL
 export IMPALA_OPENSSL_VERSION=1.0.2l
diff --git a/cmake_modules/FindZstd.cmake b/cmake_modules/FindZstd.cmake
new file mode 100644
index 0000000..e1820d3
--- /dev/null
+++ b/cmake_modules/FindZstd.cmake
@@ -0,0 +1,66 @@
+##############################################################################
+# 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 ZSTD (zstd.h, libzstd.a, libzstd.so, and libzstd.so.1)
+# ZSTD_ROOT hints the location
+#
+# This module defines
+# ZSTD_INCLUDE_DIR, directory containing headers
+# ZSTD_LIBS, directory containing zstd libraries
+# ZSTD_STATIC_LIB, path to libzstd.a
+
+set(ZSTD_SEARCH_HEADER_PATHS ${ZSTD_ROOT}/include)
+
+set(ZSTD_SEARCH_LIB_PATH ${ZSTD_ROOT}/lib)
+
+find_path(ZSTD_INCLUDE_DIR
+  NAMES zstd.h
+  PATHS ${ZSTD_SEARCH_HEADER_PATHS}
+        NO_DEFAULT_PATH
+  DOC   "Path to ZSTD headers"
+)
+
+find_library(ZSTD_LIBS NAMES zstd
+  PATHS ${ZSTD_SEARCH_LIB_PATH}
+        NO_DEFAULT_PATH
+  DOC   "Path to ZSTD library"
+)
+
+find_library(ZSTD_STATIC_LIB NAMES libzstd.a
+  PATHS ${ZSTD_SEARCH_LIB_PATH}
+        NO_DEFAULT_PATH
+  DOC   "Path to ZSTD static library"
+)
+
+if (NOT ZSTD_LIBS OR NOT ZSTD_STATIC_LIB)
+  message(FATAL_ERROR "Zstd includes and libraries NOT found. "
+    "Looked for headers in ${ZSTD_SEARCH_HEADER_PATHS}, "
+    "and for libs in ${ZSTD_SEARCH_LIB_PATH}")
+  set(ZSTD_FOUND FALSE)
+else()
+  set(ZSTD_FOUND TRUE)
+endif ()
+
+mark_as_advanced(
+  ZSTD_INCLUDE_DIR
+  ZSTD_LIBS
+  ZSTD_STATIC_LIB
+)
+
+
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index 64bd4cf..84e85f6 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -78,6 +78,7 @@ enum THdfsCompression {
   LZ4 = 8
   ZLIB = 9
   ZSTD = 10
+  BROTLI = 11
 }
 
 enum TColumnEncoding {
@@ -109,6 +110,13 @@ enum TAccessLevel {
   WRITE_ONLY = 3
 }
 
+struct TCompressionCodec {
+  // Compression codec
+  1: required THdfsCompression codec
+  // Compression level
+  2: optional i32 compression_level
+}
+
 // Mapping from names defined by Avro to values in the THdfsCompression enum.
 const map<string, THdfsCompression> COMPRESSION_MAP = {
   "": THdfsCompression.NONE,
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index ae1c5d4..85f94d5 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -111,7 +111,7 @@ struct TQueryOptions {
   7: optional i32 num_scanner_threads = 0
   11: optional string debug_action = ""
   12: optional i64 mem_limit = 0
-  14: optional CatalogObjects.THdfsCompression compression_codec
+  14: optional CatalogObjects.TCompressionCodec compression_codec
   15: optional i32 hbase_caching = 0
   16: optional bool hbase_cache_blocks = 0
   17: optional i64 parquet_file_size = 0
diff --git a/common/thrift/generate_error_codes.py b/common/thrift/generate_error_codes.py
index a21d195..590d7a8 100755
--- a/common/thrift/generate_error_codes.py
+++ b/common/thrift/generate_error_codes.py
@@ -414,6 +414,8 @@ error_codes = (
 
   ("UNAUTHORIZED_SESSION_USER", 136,
    "The user authorized on the connection '$0' does not match the session username '$1'"),
+
+  ("ZSTD_ERROR", 137, "$0 failed with error: $1"),
 )
 
 import sys
diff --git a/testdata/workloads/functional-query/queries/QueryTest/insert_parquet_multi_codecs.test b/testdata/workloads/functional-query/queries/QueryTest/insert_parquet_multi_codecs.test
new file mode 100644
index 0000000..165f04b
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert_parquet_multi_codecs.test
@@ -0,0 +1,42 @@
+====
+---- QUERY
+create table t1_default (c1 tinyint, c2 smallint, c3 int, c4 bigint, c5 boolean, c6 float,
+  c7 real, c8 double, c9 decimal(20,15), c10 timestamp, c11 char(10),
+  c13 varchar(20), c14 string) stored as parquet;
+====
+---- QUERY
+create table t1_zstd_gzip (c1 tinyint, c2 smallint, c3 int, c4 bigint, c5 boolean, c6 float,
+  c7 real, c8 double, c9 decimal(20,15), c10 timestamp, c11 char(10),
+  c13 varchar(20), c14 string) stored as parquet;
+====
+---- QUERY
+insert into t1_default select tinyint_col, smallint_col, id, bigint_col, bool_col,
+  float_col, double_col, double_col, cast(float_col as decimal(20,15)),
+  timestamp_col, cast(year as char(10)), cast(double_col as varchar(10)),
+  string_col from functional.alltypes;
+====
+---- QUERY
+insert into t1_default(c3) values (8000),(9000);
+====
+---- QUERY
+set COMPRESSION_CODEC=ZSTD;
+insert into t1_zstd_gzip select tinyint_col, smallint_col, id, bigint_col, bool_col,
+  float_col, double_col, double_col, cast(float_col as decimal(20,15)),
+  timestamp_col, cast(year as char(10)), cast(double_col as varchar(10)), string_col
+  from functional.alltypes where id < 4000;
+====
+---- QUERY
+set COMPRESSION_CODEC=GZIP;
+insert into t1_zstd_gzip(c3) values (9000);
+====
+---- QUERY
+set COMPRESSION_CODEC=GZIP;
+insert into t1_zstd_gzip select tinyint_col, smallint_col, id, bigint_col, bool_col,
+  float_col, double_col, double_col, cast(float_col as decimal(20,15)),
+  timestamp_col, cast(year as char(10)), cast(double_col as varchar(10)), string_col
+  from functional.alltypes where id >= 4000;
+====
+---- QUERY
+set COMPRESSION_CODEC=ZSTD;
+insert into t1_zstd_gzip(c3) values(8000);
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/set.test b/testdata/workloads/functional-query/queries/QueryTest/set.test
index 3acf9ac..c2756bd 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -94,7 +94,7 @@ Invalid query option: foo
 ---- QUERY
 set parquet_compression_codec=bar
 ---- CATCH
-Invalid compression codec: 'bar'. Valid values are NONE(0), DEFAULT(1), GZIP(2), DEFLATE(3), BZIP2(4), SNAPPY(5), SNAPPY_BLOCKED(6), LZO(7), LZ4(8), ZLIB(9), ZSTD(10).
+Invalid compression codec: 'bar'. Valid values are NONE(0), DEFAULT(1), GZIP(2), DEFLATE(3), BZIP2(4), SNAPPY(5), SNAPPY_BLOCKED(6), LZO(7), LZ4(8), ZLIB(9), ZSTD(10), BROTLI(11).
 ====
 ---- QUERY
 set explain_level=bar
@@ -275,4 +275,4 @@ set RM_INITIAL_MEM = "foo";
 set SCAN_NODE_CODEGEN_THRESHOLD = "foo";
 set max_io_buffers="foo";
 ---- RESULTS
-====
\ No newline at end of file
+====
diff --git a/tests/common/test_dimensions.py b/tests/common/test_dimensions.py
index bba1685..43ea337 100644
--- a/tests/common/test_dimensions.py
+++ b/tests/common/test_dimensions.py
@@ -32,7 +32,7 @@ class TableFormatInfo(object):
   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']
+  KNOWN_COMPRESSION_CODECS = ['none', 'snap', 'gzip', 'bzip', 'def', 'lzo', 'zstd']
   KNOWN_COMPRESSION_TYPES = ['none', 'block', 'record']
 
   def __init__(self, **kwargs):
diff --git a/tests/query_test/test_insert.py b/tests/query_test/test_insert.py
index 4ee8f88..07634c5 100644
--- a/tests/query_test/test_insert.py
+++ b/tests/query_test/test_insert.py
@@ -35,7 +35,7 @@ from tests.common.test_result_verifier import (
 from tests.common.test_vector import ImpalaTestDimension
 from tests.verifiers.metric_verifier import MetricVerifier
 
-PARQUET_CODECS = ['none', 'snappy', 'gzip']
+PARQUET_CODECS = ['none', 'snappy', 'gzip', 'zstd']
 
 class TestInsertQueries(ImpalaTestSuite):
   @classmethod
diff --git a/tests/query_test/test_insert_parquet.py b/tests/query_test/test_insert_parquet.py
index 19e3ae8..b207c9b 100644
--- a/tests/query_test/test_insert_parquet.py
+++ b/tests/query_test/test_insert_parquet.py
@@ -31,12 +31,13 @@ from tests.common.parametrize import UniqueDatabase
 from tests.common.skip import (SkipIfEC, SkipIfIsilon, SkipIfLocal, SkipIfS3, SkipIfABFS,
     SkipIfADLS)
 from tests.common.test_dimensions import create_exec_option_dimension
+from tests.common.test_result_verifier import verify_query_result_is_equal
 from tests.common.test_vector import ImpalaTestDimension
 from tests.util.filesystem_utils import get_fs_path
 from tests.util.get_parquet_metadata import (decode_stats_value,
     get_parquet_metadata_from_hdfs_folder)
 
-PARQUET_CODECS = ['none', 'snappy', 'gzip']
+PARQUET_CODECS = ['none', 'snappy', 'gzip', 'zstd']
 
 
 class RoundFloat():
@@ -125,6 +126,35 @@ class TestInsertParquetQueries(ImpalaTestSuite):
     self.run_test_case('insert_parquet', vector, unique_database, multiple_impalad=True)
 
 
+class TestParquetQueriesMultiCodecs(ImpalaTestSuite):
+
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestParquetQueriesMultiCodecs, cls).add_test_dimensions()
+    # Fix the exec_option vector to have a single value.
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
+        cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
+        sync_ddl=[1]))
+    cls.ImpalaTestMatrix.add_constraint(
+        lambda v: v.get_value('table_format').file_format == 'parquet')
+
+  @UniqueDatabase.parametrize(sync_ddl=True)
+  def test_insert_parquet_multi_codecs(self, vector, unique_database):
+    # Tests that parquet files are written/read correctly when using multiple codecs
+    self.run_test_case('QueryTest/insert_parquet_multi_codecs', vector, unique_database,
+        multiple_impalad=True)
+    base_table = "{0}.{1}".format(unique_database, "t1_default")
+    test_table = "{0}.{1}".format(unique_database, "t1_zstd_gzip")
+    # select all rows and compare the data in base_table and test_table
+    base_result = self.execute_query("select * from {0} order by c3".format(base_table))
+    test_result = self.execute_query("select * from {0} order by c3".format(test_table))
+    verify_query_result_is_equal(test_result.data, base_result.data)
+
+
 class TestInsertParquetInvalidCodec(ImpalaTestSuite):
 
   @classmethod


[impala] 01/06: IMPALA-8564: Add table/view create time in the lineage graph

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d9af99589fb9d8afd0844ee835b0fe945590e0bb
Author: Fredy Wijaya <fw...@cloudera.com>
AuthorDate: Tue May 21 18:01:21 2019 -0700

    IMPALA-8564: Add table/view create time in the lineage graph
    
    This patch adds table/view create time in the lineage graph. This is
    needed for Impala/Atlas integration. See ATLAS-3080.
    
    Below is an example of the updated lineage graph.
    {
        "queryText":"create table lineage_test_tbl as select int_col, tinyint_col from functional.alltypes",
        "queryId":"0:0",
        "hash":"407f23b24758ffcb2ac445b9703f5c44",
        "user":"dummy_user",
        "timestamp":1547867921,
        "edges":[
            {
                "sources":[
                    1
                ],
                "targets":[
                    0
                ],
                "edgeType":"PROJECTION"
            },
            {
                "sources":[
                    3
                ],
                "targets":[
                    2
                ],
                "edgeType":"PROJECTION"
            }
        ],
        "vertices":[
            {
                "id":0,
                "vertexType":"COLUMN",
                "vertexId":"int_col",
                "metadata":{
                    "tableName":"default.lineage_test_tbl",
                    "tableCreateTime":1559151337
                }
            },
            {
                "id":1,
                "vertexType":"COLUMN",
                "vertexId":"functional.alltypes.int_col",
                "metadata":{
                    "tableName":"functional.alltypes",
                    "tableCreateTime":1559151317
                }
            },
            {
                "id":2,
                "vertexType":"COLUMN",
                "vertexId":"tinyint_col",
                "metadata":{
                    "tableName":"default.lineage_test_tbl",
                    "tableCreateTime":1559151337
                }
            },
            {
                "id":3,
                "vertexType":"COLUMN",
                "vertexId":"functional.alltypes.tinyint_col",
                "metadata":{
                    "tableName":"functional.alltypes",
                    "tableCreateTime":1559151317
                }
            }
        ]
    }
    
    Testing:
    - Updated lineage tests in PlannerTest
    - Updated test_lineage.py
    - Ran all FE tests
    
    Change-Id: If4f578d7b299a76c30323b10a883ba32f8713d82
    Reviewed-on: http://gerrit.cloudera.org:8080/13399
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/client-request-state.h              |    8 +-
 be/src/service/frontend.cc                         |    3 +-
 be/src/service/impala-server.cc                    |   16 +
 be/src/util/lineage-util.h                         |    9 +
 common/thrift/CatalogService.thrift                |    8 +
 common/thrift/LineageGraph.thrift                  |   15 +-
 .../apache/impala/analysis/ColumnLineageGraph.java |  665 ++++---
 .../impala/analysis/CreateOrAlterViewStmtBase.java |    5 +-
 .../impala/catalog/CatalogServiceCatalog.java      |    1 -
 .../java/org/apache/impala/planner/Planner.java    |   10 +-
 .../apache/impala/service/CatalogOpExecutor.java   |    7 +-
 .../org/apache/impala/service/JniFrontend.java     |    1 -
 .../org/apache/impala/planner/PlannerTestBase.java |    8 +-
 .../queries/PlannerTest/lineage.test               | 2004 ++++++++++++++++----
 tests/custom_cluster/test_lineage.py               |   38 +-
 15 files changed, 2181 insertions(+), 617 deletions(-)

diff --git a/be/src/service/client-request-state.h b/be/src/service/client-request-state.h
index ae3dc95..9347a08 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -197,7 +197,7 @@ class ClientRequestState {
   void set_fetched_rows() { fetched_rows_ = true; }
   bool fetched_rows() const { return fetched_rows_; }
   bool returns_result_set() { return !result_metadata_.columns.empty(); }
-  const TResultSetMetadata* result_metadata() { return &result_metadata_; }
+  const TResultSetMetadata* result_metadata() const { return &result_metadata_; }
   const TUniqueId& query_id() const { return query_ctx_.query_id; }
   const TExecRequest& exec_request() const { return exec_request_; }
   TStmtType::type stmt_type() const { return exec_request_.stmt_type; }
@@ -259,7 +259,11 @@ class ClientRequestState {
   RuntimeProfile::EventSequence* query_events() const { return query_events_; }
   RuntimeProfile* summary_profile() { return summary_profile_; }
 
- protected:
+  /// Returns nullptr when catalog_op_type is not DDL.
+  const TDdlExecResponse* ddl_exec_response() const {
+    return catalog_op_executor_->ddl_exec_response();
+  }
+protected:
   /// Updates the end_time_us_ of this query if it isn't set. The end time is determined
   /// when this function is called for the first time, calling it multiple times does not
   /// change the end time.
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index 6ef000a..757ef92 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -298,5 +298,4 @@ Status Frontend::BuildTestDescriptorTable(const TBuildTestDescriptorTableParams&
 // Call FE post-query execution hook
 Status Frontend::CallQueryCompleteHooks(const TQueryCompleteContext& context) {
   return JniUtil::CallJniMethod(fe_, call_query_complete_hooks_id_, context);
-}
-
+}
\ No newline at end of file
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index eb6bec7..9fb58fe 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -480,6 +480,22 @@ Status ImpalaServer::LogLineageRecord(const ClientRequestState& client_request_s
   } else {
     return Status::OK();
   }
+
+  if (client_request_state.catalog_op_type() == TCatalogOpType::DDL) {
+    const TDdlExecResponse* ddl_exec_response = client_request_state.ddl_exec_response();
+    // Update vertices that have -1 table_create_time for a newly created table/view.
+    if (ddl_exec_response->__isset.table_name &&
+        ddl_exec_response->__isset.table_create_time) {
+      for (auto &vertex: lineage_graph.vertices) {
+        if (!vertex.__isset.metadata) continue;
+        if (vertex.metadata.table_name == ddl_exec_response->table_name &&
+            vertex.metadata.table_create_time == -1) {
+          vertex.metadata.__set_table_create_time(ddl_exec_response->table_create_time);
+        }
+      }
+    }
+  }
+
   // Set the query end time in TLineageGraph. Must use UNIX time directly rather than
   // e.g. converting from client_request_state.end_time() (IMPALA-4440).
   lineage_graph.__set_ended(UnixMillis() / 1000);
diff --git a/be/src/util/lineage-util.h b/be/src/util/lineage-util.h
index e52bb4c..f5dd722 100644
--- a/be/src/util/lineage-util.h
+++ b/be/src/util/lineage-util.h
@@ -42,6 +42,15 @@ class LineageUtil {
       writer->String("COLUMN");
       writer->String("vertexId");
       writer->String(vertex.label.c_str());
+      if (vertex.__isset.metadata) {
+        writer->String("metadata");
+        writer->StartObject();
+        writer->String("tableName");
+        writer->String(vertex.metadata.table_name.c_str());
+        writer->String("tableCreateTime");
+        writer->Int64(vertex.metadata.table_create_time);
+        writer->EndObject();
+      }
       writer->EndObject();
     }
 
diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift
index f970d17..f76e2c3 100644
--- a/common/thrift/CatalogService.thrift
+++ b/common/thrift/CatalogService.thrift
@@ -164,6 +164,14 @@ struct TDdlExecResponse {
   // Result of DDL operation to be returned to the client. Currently only set
   // by COMPUTE STATS and ALTER TABLE.
   3: optional Results.TResultSet result_set
+
+  // The table/view name in HMS. Set only for CREATE TABLE, CREATE TABLE AS SELECT,
+  // CREATE TABLE LIKE, and CREATE VIEW statements.
+  4: optional string table_name
+
+  // The table/view create time stored in HMS. Set only for CREATE TABLE,
+  // CREATE TABLE AS SELECT, CREATE TABLE LIKE, and CREATE VIEW statements.
+  5: optional i64 table_create_time
 }
 
 // Updates the metastore with new partition information and returns a response
diff --git a/common/thrift/LineageGraph.thrift b/common/thrift/LineageGraph.thrift
index b1d64b6..51e0493 100644
--- a/common/thrift/LineageGraph.thrift
+++ b/common/thrift/LineageGraph.thrift
@@ -20,17 +20,28 @@ namespace java org.apache.impala.thrift
 
 include "Types.thrift"
 
+struct TVertexMetadata {
+  // Table name.
+  1: required string table_name
+
+  // Create time of the table/view.
+  2: required i64 table_create_time
+}
+
 struct TVertex {
   // Vertex id
   1: required i64 id
 
   // Column label
   2: required string label
+
+  // Metadata of the vertex.
+  3: optional TVertexMetadata metadata
 }
 
 enum TEdgeType {
-    PROJECTION = 0
-    PREDICATE = 1
+  PROJECTION = 0
+  PREDICATE = 1
 }
 
 struct TMultiEdge {
diff --git a/fe/src/main/java/org/apache/impala/analysis/ColumnLineageGraph.java b/fe/src/main/java/org/apache/impala/analysis/ColumnLineageGraph.java
index 53d3a3e..6285af6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ColumnLineageGraph.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ColumnLineageGraph.java
@@ -21,12 +21,18 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Set;
 import java.util.TreeSet;
 
+import com.google.common.collect.ImmutableSortedMap;
+import org.apache.impala.analysis.ColumnLineageGraph.Vertex.Metadata;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.common.Id;
 import org.apache.impala.common.IdGenerator;
@@ -36,6 +42,7 @@ import org.apache.impala.thrift.TMultiEdge;
 import org.apache.impala.thrift.TQueryCtx;
 import org.apache.impala.thrift.TUniqueId;
 import org.apache.impala.thrift.TVertex;
+import org.apache.impala.thrift.TVertexMetadata;
 import org.apache.impala.util.TUniqueIdUtil;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
@@ -54,238 +61,368 @@ import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
 
 /**
- * Represents a vertex in the column lineage graph. A Vertex may correspond to a base
- * table column, a column in the destination table (for the case of INSERT or CTAS
- * queries) or a result expr (labeled column of a query result set).
+ * Represents the column lineage graph of a query. This is a directional graph that is
+ * used to track dependencies among the table/column entities that participate in
+ * a query. There are two types of dependencies that are represented as edges in the
+ * column lineage graph:
+ * a) Projection dependency: This is a dependency between a set of source
+ * columns (base table columns) and a single target (result expr or table column).
+ * This dependency indicates that values of the target depend on the values of the source
+ * columns.
+ * b) Predicate dependency: This is a dependency between a set of target
+ * columns (or exprs) and a set of source columns (base table columns). It indicates that
+ * the source columns restrict the values of their targets (e.g. by participating in
+ * WHERE clause predicates).
+ *
+ * The following dependencies are generated for a query:
+ * - Exactly one projection dependency for every result expr / target column.
+ * - Exactly one predicate dependency that targets all result exprs / target cols and
+ *   depends on all columns participating in a conjunct in the query.
+ * - Special case of analytic fns: One predicate dependency per result expr / target col
+ *   whose value is directly or indirectly affected by an analytic function with a
+ *   partition by and/or order by clause.
  */
-final class Vertex implements Comparable<Vertex> {
-  // Unique identifier of this vertex.
-  private final VertexId id_;
-
-  private final String type_ = "COLUMN";
-
-  // A fully-qualified column name or the label of a result expr
-  private final String label_;
-
-  public Vertex(VertexId id, String label) {
-    Preconditions.checkNotNull(id);
-    Preconditions.checkNotNull(label);
-    id_ = id;
-    label_ = label;
-  }
-  public VertexId getVertexId() { return id_; }
-  public String getLabel() { return label_; }
-  public String getType() { return type_; }
-
-  @Override
-  public String toString() { return "(" + id_ + ":" + type_ + ":" + label_ + ")"; }
-
+public class ColumnLineageGraph {
   /**
-   * Encodes this Vertex object into a JSON object represented by a Map.
+   * Represents a vertex in the column lineage graph. A Vertex may correspond to a base
+   * table column, a column in the destination table (for the case of INSERT or CTAS
+   * queries) or a result expr (labeled column of a query result set).
    */
-  public Map<String,Object> toJson() {
-    // Use a LinkedHashMap to generate a strict ordering of elements.
-    Map<String,Object> obj = new LinkedHashMap<>();
-    obj.put("id", id_.asInt());
-    obj.put("vertexType", type_);
-    obj.put("vertexId", label_);
-    return obj;
-  }
+  public static final class Vertex implements Comparable<Vertex> {
+    public static class Metadata {
+      private final String tableName_; // the table name
+      private final long tableCreateTime_; // the table/view create time
+
+      public Metadata(String tableName, long tableCreateTime) {
+        tableName_ = tableName;
+        tableCreateTime_ = tableCreateTime;
+      }
 
-  /**
-   * Constructs a Vertex object from a JSON object. The new object is returned.
-   */
-  public static Vertex fromJsonObj(JSONObject obj) {
-    int id = ((Long) obj.get("id")).intValue();
-    String label = (String) obj.get("vertexId");
-    return new Vertex(new VertexId(id), label);
-  }
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        Metadata metadata = (Metadata) o;
+        return tableCreateTime_ == metadata.tableCreateTime_ &&
+            Objects.equals(tableName_, metadata.tableName_);
+      }
 
-  /**
-   * Encodes this Vertex object into a thrift object
-   */
-  public TVertex toThrift() {
-    return new TVertex(id_.asInt(), label_);
-  }
+      @Override
+      public int hashCode() {
+        return Objects.hash(tableName_, tableCreateTime_);
+      }
 
-  /**
-   * Constructs a Vertex object from a thrift object.
-   */
-  public static Vertex fromThrift(TVertex vertex) {
-    int id = ((Long) vertex.id).intValue();
-    return new Vertex(new VertexId(id), vertex.label);
-  }
+      /**
+       * For testing only. We ignore the table create time.
+       */
+      private boolean equalsForTests(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        Metadata metadata = (Metadata) o;
+        return Objects.equals(tableName_, metadata.tableName_);
+      }
+    }
 
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) return false;
-    if (obj.getClass() != this.getClass()) return false;
-    Vertex vertex = (Vertex) obj;
-    return this.id_.equals(vertex.id_);
-  }
+    // Unique identifier of this vertex.
+    private final VertexId id_;
+
+    private final String type_ = "COLUMN";
+
+    // A fully-qualified column name or the label of a result expr
+    private final String label_;
+
+    // The table metadata.
+    private final Metadata metadata_;
+
+    public Vertex(VertexId id, String label, Metadata metadata) {
+      Preconditions.checkNotNull(id);
+      Preconditions.checkNotNull(label);
+      id_ = id;
+      label_ = label;
+      metadata_ = metadata;
+    }
+    public VertexId getVertexId() { return id_; }
+    public String getLabel() { return label_; }
+    public String getType() { return type_; }
+    public Metadata getMetadata() { return metadata_; }
+
+    @Override
+    public String toString() { return "(" + id_ + ":" + type_ + ":" + label_ + ")"; }
+
+    /**
+     * Encodes this Vertex object into a JSON object represented by a Map.
+     */
+    public Map<String, Object> toJson() {
+      // Use a LinkedHashMap to generate a strict ordering of elements.
+      Map<String,Object> obj = new LinkedHashMap<>();
+      obj.put("id", id_.asInt());
+      obj.put("vertexType", type_);
+      obj.put("vertexId", label_);
+      if (metadata_ != null) {
+        JSONObject jsonMetadata = new JSONObject();
+        jsonMetadata.put("tableName", metadata_.tableName_);
+        jsonMetadata.put("tableCreateTime", metadata_.tableCreateTime_);
+        obj.put("metadata", jsonMetadata);
+      }
+      return obj;
+    }
+
+    /**
+     * Constructs a Vertex object from a JSON object. The new object is returned.
+     */
+    public static Vertex fromJsonObj(JSONObject obj) {
+      int id = ((Long) obj.get("id")).intValue();
+      String label = (String) obj.get("vertexId");
+      JSONObject jsonMetadata = (JSONObject) obj.get("metadata");
+      if (jsonMetadata == null) {
+        return new Vertex(new VertexId(id), label, null);
+      }
+      String tableName = (String) jsonMetadata.get("tableName");
+      long tableCreateTime = (Long) jsonMetadata.get("tableCreateTime");
+      return new Vertex(new VertexId(id), label, new Metadata(tableName,
+          tableCreateTime));
+    }
+
+    /**
+     * Encodes this Vertex object into a thrift object
+     */
+    public TVertex toThrift() {
+      TVertex vertex = new TVertex(id_.asInt(), label_);
+      if (metadata_ != null) {
+        TVertexMetadata metadata = new TVertexMetadata(metadata_.tableName_,
+            metadata_.tableCreateTime_);
+        vertex.setMetadata(metadata);
+      }
+      return vertex;
+    }
+
+    /**
+     * Constructs a Vertex object from a thrift object.
+     */
+    public static Vertex fromThrift(TVertex vertex) {
+      int id = ((Long) vertex.id).intValue();
+      TVertexMetadata thriftMetadata = vertex.getMetadata();
+      Metadata metadata = null;
+      if (thriftMetadata != null) {
+        metadata = new Metadata(thriftMetadata.getTable_name(),
+            thriftMetadata.getTable_create_time());
+      }
+      return new Vertex(new VertexId(id), vertex.label, metadata);
+    }
 
-  @Override
-  public int compareTo(Vertex cmp) { return this.id_.compareTo(cmp.id_); }
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      Vertex vertex = (Vertex) o;
+      return Objects.equals(id_, vertex.id_) &&
+          Objects.equals(type_, vertex.type_) &&
+          Objects.equals(label_, vertex.label_) &&
+          Objects.equals(metadata_, vertex.metadata_);
+    }
 
-  @Override
-  public int hashCode() { return id_.hashCode(); }
-}
+    @Override
+    public int hashCode() {
+      return Objects.hash(id_, type_, label_, metadata_);
+    }
 
-/**
- * Represents the unique identifier of a Vertex.
- */
-class VertexId extends Id<VertexId> {
-  protected VertexId(int id) {
-    super(id);
-  }
-  public static IdGenerator<VertexId> createGenerator() {
-    return new IdGenerator<VertexId>() {
-      @Override
-      public VertexId getNextId() { return new VertexId(nextId_++); }
-      @Override
-      public VertexId getMaxId() { return new VertexId(nextId_ - 1); }
-    };
-  }
-}
+    /**
+     * For testing only.
+     */
+    private boolean equalsForTests(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      Vertex vertex = (Vertex) o;
+      return Objects.equals(id_, vertex.id_) &&
+          ((metadata_ == vertex.metadata_) ||
+              metadata_ != null && metadata_.equalsForTests(vertex.metadata_));
+    }
 
-/**
- * Represents a set of uni-directional edges in the column lineage graph, one edge from
- * every source Vertex in 'sources_' to every target Vertex in 'targets_'. An edge
- * indicates a dependency between a source and a target Vertex. There are two types of
- * edges, PROJECTION and PREDICATE, that are described in the ColumnLineageGraph class.
- */
-final class MultiEdge {
-  public static enum EdgeType {
-    PROJECTION, PREDICATE
-  }
-  private final Set<Vertex> sources_;
-  private final Set<Vertex> targets_;
-  private final EdgeType edgeType_;
-
-  public MultiEdge(Set<Vertex> sources, Set<Vertex> targets, EdgeType type) {
-    sources_ = sources;
-    targets_ = targets;
-    edgeType_ = type;
+    @Override
+    public int compareTo(Vertex cmp) { return this.id_.compareTo(cmp.id_); }
   }
 
   /**
-   * Return an ordered set of source vertices.
+   * Represents the unique identifier of a Vertex.
    */
-  private ImmutableSortedSet<Vertex> getOrderedSources() {
-    return ImmutableSortedSet.copyOf(sources_);
+  private static final class VertexId extends Id<VertexId> {
+    protected VertexId(int id) {
+      super(id);
+    }
+    public static IdGenerator<VertexId> createGenerator() {
+      return new IdGenerator<VertexId>() {
+        @Override
+        public VertexId getNextId() { return new VertexId(nextId_++); }
+        @Override
+        public VertexId getMaxId() { return new VertexId(nextId_ - 1); }
+      };
+    }
   }
 
   /**
-   * Return an ordered set of target vertices.
+   * Represents a set of uni-directional edges in the column lineage graph, one edge from
+   * every source Vertex in 'sources_' to every target Vertex in 'targets_'. An edge
+   * indicates a dependency between a source and a target Vertex. There are two types of
+   * edges, PROJECTION and PREDICATE, that are described in the ColumnLineageGraph class.
    */
-  private ImmutableSortedSet<Vertex> getOrderedTargets() {
-    return ImmutableSortedSet.copyOf(targets_);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    Joiner joiner = Joiner.on(",");
-    builder.append("Sources: [");
-    builder.append(joiner.join(getOrderedSources()) + "]\n");
-    builder.append("Targets: [");
-    builder.append(joiner.join(getOrderedTargets()) + "]\n");
-    builder.append("Type: " + edgeType_);
-    return builder.toString();
-  }
+  private static final class MultiEdge {
+    public static enum EdgeType {
+      PROJECTION, PREDICATE
+    }
+    private final Set<Vertex> sources_;
+    private final Set<Vertex> targets_;
+    private final EdgeType edgeType_;
+
+    public MultiEdge(Set<Vertex> sources, Set<Vertex> targets, EdgeType type) {
+      sources_ = sources;
+      targets_ = targets;
+      edgeType_ = type;
+    }
+
+    /**
+     * Return an ordered set of source vertices.
+     */
+    private ImmutableSortedSet<Vertex> getOrderedSources() {
+      return ImmutableSortedSet.copyOf(sources_);
+    }
+
+    /**
+     * Return an ordered set of target vertices.
+     */
+    private ImmutableSortedSet<Vertex> getOrderedTargets() {
+      return ImmutableSortedSet.copyOf(targets_);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      Joiner joiner = Joiner.on(",");
+      builder.append("Sources: [");
+      builder.append(joiner.join(getOrderedSources()) + "]\n");
+      builder.append("Targets: [");
+      builder.append(joiner.join(getOrderedTargets()) + "]\n");
+      builder.append("Type: " + edgeType_);
+      return builder.toString();
+    }
+
+    /**
+     * Encodes this MultiEdge object to a JSON object represented by a Map.
+     * Returns a LinkedHashMap to guarantee a consistent ordering of elements.
+     */
+    public Map<String,Object> toJson() {
+      Map<String,Object> obj = new LinkedHashMap<>();
+      // Add sources
+      JSONArray sourceIds = new JSONArray();
+      for (Vertex vertex: getOrderedSources()) {
+        sourceIds.add(vertex.getVertexId());
+      }
+      obj.put("sources", sourceIds);
+      // Add targets
+      JSONArray targetIds = new JSONArray();
+      for (Vertex vertex: getOrderedTargets()) {
+        targetIds.add(vertex.getVertexId());
+      }
+      obj.put("targets", targetIds);
+      obj.put("edgeType", edgeType_.toString());
+      return obj;
+    }
+
+    /**
+     * Encodes this MultiEdge object to a thrift object
+     */
+    public TMultiEdge toThrift() {
+      List<TVertex> sources = new ArrayList<>();
+      for (Vertex vertex: getOrderedSources()) {
+        sources.add(vertex.toThrift());
+      }
+      List<TVertex> targets = new ArrayList<>();
+      for (Vertex vertex: getOrderedTargets()) {
+        targets.add(vertex.toThrift());
+      }
+      if (edgeType_ == EdgeType.PROJECTION) {
+        return new TMultiEdge(sources, targets, TEdgeType.PROJECTION);
+      }
+      return new TMultiEdge(sources, targets, TEdgeType.PREDICATE);
+    }
 
-  /**
-   * Encodes this MultiEdge object to a JSON object represented by a Map.
-   * Returns a LinkedHashMap to guarantee a consistent ordering of elements.
-   */
-  public Map<String,Object> toJson() {
-    Map<String,Object> obj = new LinkedHashMap<>();
-    // Add sources
-    JSONArray sourceIds = new JSONArray();
-    for (Vertex vertex: getOrderedSources()) {
-      sourceIds.add(vertex.getVertexId());
-    }
-    obj.put("sources", sourceIds);
-    // Add targets
-    JSONArray targetIds = new JSONArray();
-    for (Vertex vertex: getOrderedTargets()) {
-      targetIds.add(vertex.getVertexId());
-    }
-    obj.put("targets", targetIds);
-    obj.put("edgeType", edgeType_.toString());
-    return obj;
-  }
+    /**
+     * Constructs a MultiEdge object from a thrift object.
+     */
+    public static MultiEdge fromThrift(TMultiEdge obj) {
+      Set<Vertex> sources = new HashSet<>();
+      for (TVertex vertex: obj.sources) {
+        sources.add(Vertex.fromThrift(vertex));
+      }
+      Set<Vertex> targets = new HashSet<>();
+      for (TVertex vertex: obj.targets) {
+        targets.add(Vertex.fromThrift(vertex));
+      }
+      if (obj.edgetype == TEdgeType.PROJECTION) {
+        return new MultiEdge(sources, targets, EdgeType.PROJECTION);
+      }
+      return new MultiEdge(sources, targets, EdgeType.PREDICATE);
+    }
 
-  /**
-   * Encodes this MultiEdge object to a thrift object
-   */
-  public TMultiEdge toThrift() {
-    List<TVertex> sources = new ArrayList<>();
-    for (Vertex vertex: getOrderedSources()) {
-      sources.add(vertex.toThrift());
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      MultiEdge multiEdge = (MultiEdge) o;
+      return Objects.equals(sources_, multiEdge.sources_) &&
+          Objects.equals(targets_, multiEdge.targets_) &&
+          edgeType_ == multiEdge.edgeType_;
     }
-    List<TVertex> targets = new ArrayList<>();
-    for (Vertex vertex: getOrderedTargets()) {
-      targets.add(vertex.toThrift());
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(sources_, targets_, edgeType_);
     }
-    if (edgeType_ == EdgeType.PROJECTION) {
-      return new TMultiEdge(sources, targets, TEdgeType.PROJECTION);
+
+    /**
+     * For testing only.
+     */
+    private boolean equalsForTests(Object obj) {
+      if (obj == null) return false;
+      if (obj.getClass() != this.getClass()) return false;
+      MultiEdge edge = (MultiEdge) obj;
+      return setEqualsForTests(edge.sources_, this.sources_) &&
+          setEqualsForTests(edge.targets_, this.targets_) &&
+          edge.edgeType_ == this.edgeType_;
     }
-    return new TMultiEdge(sources, targets, TEdgeType.PREDICATE);
   }
 
-  /**
-   * Constructs a MultiEdge object from a thrift object.
-   */
-  public static MultiEdge fromThrift(TMultiEdge obj){
-    Set<Vertex> sources = new HashSet<>();
-    for (TVertex vertex: obj.sources) {
-      sources.add(Vertex.fromThrift(vertex));
+  public static class ColumnLabel implements Comparable<ColumnLabel> {
+    private final String columnLabel_;
+    private final TableName tableName_;
+
+    public ColumnLabel(String columnName, TableName tableName) {
+      columnLabel_ = columnName;
+      tableName_ = tableName;
     }
-    Set<Vertex> targets = new HashSet<>();
-    for (TVertex vertex: obj.targets) {
-      targets.add(Vertex.fromThrift(vertex));
+
+    public ColumnLabel(String columnName) {
+      this(columnName, null);
     }
-    if (obj.edgetype == TEdgeType.PROJECTION) {
-      return new MultiEdge(sources, targets, EdgeType.PROJECTION);
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      ColumnLabel that = (ColumnLabel) o;
+      return Objects.equals(columnLabel_, that.columnLabel_);
     }
-    return new MultiEdge(sources, targets, EdgeType.PREDICATE);
-  }
 
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) return false;
-    if (obj.getClass() != this.getClass()) return false;
-    MultiEdge edge = (MultiEdge) obj;
-    return edge.sources_.equals(this.sources_) &&
-        edge.targets_.equals(this.targets_) &&
-        edge.edgeType_ == this.edgeType_;
+    @Override
+    public int hashCode() {
+      return Objects.hash(columnLabel_);
+    }
+
+    @Override
+    public int compareTo(ColumnLabel o) {
+      return columnLabel_.compareTo(o.columnLabel_);
+    }
   }
-}
 
-/**
- * Represents the column lineage graph of a query. This is a directional graph that is
- * used to track dependencies among the table/column entities that participate in
- * a query. There are two types of dependencies that are represented as edges in the
- * column lineage graph:
- * a) Projection dependency: This is a dependency between a set of source
- * columns (base table columns) and a single target (result expr or table column).
- * This dependency indicates that values of the target depend on the values of the source
- * columns.
- * b) Predicate dependency: This is a dependency between a set of target
- * columns (or exprs) and a set of source columns (base table columns). It indicates that
- * the source columns restrict the values of their targets (e.g. by participating in
- * WHERE clause predicates).
- *
- * The following dependencies are generated for a query:
- * - Exactly one projection dependency for every result expr / target column.
- * - Exactly one predicate dependency that targets all result exprs / target cols and
- *   depends on all columns participating in a conjunct in the query.
- * - Special case of analytic fns: One predicate dependency per result expr / target col
- *   whose value is directly or indirectly affected by an analytic function with a
- *   partition by and/or order by clause.
- */
-public class ColumnLineageGraph {
   private final static Logger LOG = LoggerFactory.getLogger(ColumnLineageGraph.class);
   // Query statement
   private String queryStr_;
@@ -312,7 +449,7 @@ public class ColumnLineageGraph {
   // For an INSERT or a CTAS, these are the columns of the
   // destination table plus any partitioning columns (when dynamic partitioning is used).
   // For a SELECT stmt, they are the labels of the result exprs.
-  private final List<String> targetColumnLabels_ = new ArrayList<>();
+  private final List<ColumnLabel> targetColumnLabels_ = new ArrayList<>();
 
   // Repository for tuple and slot descriptors for this query. Use it to construct the
   // column lineage graph.
@@ -345,17 +482,35 @@ public class ColumnLineageGraph {
    * 'targets' labels (representing column names or result expr labels). The new
    * MultiEdge object is returned.
    */
-  private MultiEdge createMultiEdge(Set<String> targets, Set<String> sources,
-      MultiEdge.EdgeType type) {
+  private MultiEdge createMultiEdge(Set<ColumnLabel> targets,
+      Map<String, SlotDescriptor> sources, MultiEdge.EdgeType type, Analyzer analyzer) {
     // createVertex() generates new IDs; we sort the input sets to make the output
     // deterministic and independent of the ordering of the input sets.
     Set<Vertex> targetVertices = new HashSet<>();
-    for (String target: ImmutableSortedSet.copyOf(targets)) {
-      targetVertices.add(createVertex(target));
+    for (ColumnLabel target: ImmutableSortedSet.copyOf(targets)) {
+      Metadata metadata = null;
+      if (target.tableName_ != null) {
+        FeTable feTable = analyzer.getStmtTableCache().tables.get(target.tableName_);
+        if (feTable != null && feTable.getMetaStoreTable() != null) {
+          metadata = new Metadata(target.tableName_.toString(),
+              feTable.getMetaStoreTable().getCreateTime());
+        } else {
+          // -1 is just a placeholder that will be updated after the table/view has been
+          // created. See impala-server.cc (LogLineageRecord) for more information.
+          metadata = new Metadata(target.tableName_.toString(), -1);
+        }
+      }
+      targetVertices.add(createVertex(target.columnLabel_, metadata));
     }
     Set<Vertex> sourceVertices = new HashSet<>();
-    for (String source: ImmutableSortedSet.copyOf(sources)) {
-      sourceVertices.add(createVertex(source));
+    for (Map.Entry<String, SlotDescriptor> source:
+        ImmutableSortedMap.copyOf(sources).entrySet()) {
+      FeTable feTable = source.getValue().getParent().getTable();
+      Preconditions.checkState(feTable != null);
+      Metadata metadata = feTable != null && feTable.getMetaStoreTable() != null ?
+          new Metadata(feTable.getTableName().toString(),
+              feTable.getMetaStoreTable().getCreateTime()) : null;
+      sourceVertices.add(createVertex(source.getKey(), metadata));
     }
     MultiEdge edge = new MultiEdge(sourceVertices, targetVertices, type);
     edges_.add(edge);
@@ -366,10 +521,10 @@ public class ColumnLineageGraph {
    * Creates a new vertex in the column lineage graph. The new Vertex object is
    * returned. If a Vertex with the same label already exists, reuse it.
    */
-  private Vertex createVertex(String label) {
+  private Vertex createVertex(String label, Metadata metadata) {
     Vertex newVertex = vertices_.get(label);
     if (newVertex != null) return newVertex;
-    newVertex = new Vertex(vertexIdGenerator.getNextId(), label);
+    newVertex = new Vertex(vertexIdGenerator.getNextId(), label, metadata);
     vertices_.put(newVertex.getLabel(), newVertex);
     idToVertexMap_.put(newVertex.getVertexId(), newVertex);
     return newVertex;
@@ -381,7 +536,7 @@ public class ColumnLineageGraph {
    */
   public void computeLineageGraph(List<Expr> resultExprs, Analyzer rootAnalyzer) {
     init(rootAnalyzer);
-    computeProjectionDependencies(resultExprs);
+    computeProjectionDependencies(resultExprs, rootAnalyzer);
     computeResultPredicateDependencies(rootAnalyzer);
   }
 
@@ -404,28 +559,29 @@ public class ColumnLineageGraph {
     queryId_ = queryCtx.query_id;
   }
 
-  private void computeProjectionDependencies(List<Expr> resultExprs) {
+  private void computeProjectionDependencies(List<Expr> resultExprs, Analyzer analyzer) {
     Preconditions.checkNotNull(resultExprs);
     Preconditions.checkState(!resultExprs.isEmpty());
     Preconditions.checkState(resultExprs.size() == targetColumnLabels_.size());
     for (int i = 0; i < resultExprs.size(); ++i) {
       Expr expr = resultExprs.get(i);
-      Set<String> sourceBaseCols = new HashSet<>();
+      Map<String, SlotDescriptor> sourceBaseCols = new HashMap<>();
       List<Expr> dependentExprs = new ArrayList<>();
       getSourceBaseCols(expr, sourceBaseCols, dependentExprs, false);
-      Set<String> targets = Sets.newHashSet(targetColumnLabels_.get(i));
-      createMultiEdge(targets, sourceBaseCols, MultiEdge.EdgeType.PROJECTION);
+      Set<ColumnLabel> targets = Sets.newHashSet(targetColumnLabels_.get(i));
+      createMultiEdge(targets, sourceBaseCols, MultiEdge.EdgeType.PROJECTION, analyzer);
       if (!dependentExprs.isEmpty()) {
         // We have additional exprs that 'expr' has a predicate dependency on.
         // Gather the transitive predicate dependencies of 'expr' based on its direct
         // predicate dependencies. For each direct predicate dependency p, 'expr' is
         // transitively predicate dependent on all exprs that p is projection and
         // predicate dependent on.
-        Set<String> predicateBaseCols = new HashSet<>();
+        Map<String, SlotDescriptor> predicateBaseCols = new HashMap<>();
         for (Expr dependentExpr: dependentExprs) {
           getSourceBaseCols(dependentExpr, predicateBaseCols, null, true);
         }
-        createMultiEdge(targets, predicateBaseCols, MultiEdge.EdgeType.PREDICATE);
+        createMultiEdge(targets, predicateBaseCols, MultiEdge.EdgeType.PREDICATE,
+            analyzer);
       }
     }
   }
@@ -441,13 +597,13 @@ public class ColumnLineageGraph {
       if (expr.isAuxExpr()) continue;
       resultDependencyPredicates_.add(expr);
     }
-    Set<String> predicateBaseCols = new HashSet<>();
+    Map<String, SlotDescriptor> predicateBaseCols = new HashMap<>();
     for (Expr expr: resultDependencyPredicates_) {
       getSourceBaseCols(expr, predicateBaseCols, null, true);
     }
     if (predicateBaseCols.isEmpty()) return;
-    Set<String> targets = Sets.newHashSet(targetColumnLabels_);
-    createMultiEdge(targets, predicateBaseCols, MultiEdge.EdgeType.PREDICATE);
+    Set<ColumnLabel> targets = Sets.newHashSet(targetColumnLabels_);
+    createMultiEdge(targets, predicateBaseCols, MultiEdge.EdgeType.PREDICATE, analyzer);
   }
 
   /**
@@ -460,7 +616,7 @@ public class ColumnLineageGraph {
    * connected to. Which children are filtered depends on the type of 'expr' (e.g. for
    * AnalyticFunctionExpr, grouping and sorting exprs are filtered out).
    */
-  private void getSourceBaseCols(Expr expr, Set<String> sourceBaseCols,
+  private void getSourceBaseCols(Expr expr, Map<String, SlotDescriptor> sourceBaseCols,
       List<Expr> directPredDeps, boolean traversePredDeps) {
     List<Expr> exprsToTraverse = getProjectionDeps(expr);
     List<Expr> predicateDepExprs = getPredicateDeps(expr);
@@ -478,7 +634,7 @@ public class ColumnLineageGraph {
         // slot should correspond to a materialized tuple of a table
         Preconditions.checkState(slotDesc.getParent().isMaterialized());
         List<String> path = slotDesc.getPath().getCanonicalPath();
-        sourceBaseCols.add(Joiner.on(".").join(path));
+        sourceBaseCols.put(Joiner.on(".").join(path), slotDesc);
       } else {
         for (Expr sourceExpr: sourceExprs) {
           getSourceBaseCols(sourceExpr, sourceBaseCols, directPredDeps,
@@ -586,9 +742,11 @@ public class ColumnLineageGraph {
   public static ColumnLineageGraph fromThrift(TLineageGraph obj) {
     ColumnLineageGraph lineage =
         new ColumnLineageGraph(obj.query_text, obj.query_id, obj.user, obj.started);
+    Map<TVertex, Vertex> vertexMap = new HashMap<>();
     TreeSet<Vertex> vertices = Sets.newTreeSet();
     for (TVertex vertex: obj.vertices) {
-      vertices.add(Vertex.fromThrift(vertex));
+      Vertex v = Vertex.fromThrift(vertex);
+      vertices.add(v);
     }
     lineage.setVertices(vertices);
     for (TMultiEdge edge: obj.edges) {
@@ -663,18 +821,68 @@ public class ColumnLineageGraph {
     return vertices;
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  /**
+   * This is only for testing. It does not check for user and timestamp fields.
+   */
+  public boolean equalsForTests(Object obj) {
     if (obj == null) return false;
     if (obj.getClass() != this.getClass()) return false;
     ColumnLineageGraph g = (ColumnLineageGraph) obj;
-    if (!this.vertices_.equals(g.vertices_) ||
-        !this.edges_.equals(g.edges_)) {
+    if (!mapEqualsForTests(this.vertices_, g.vertices_) ||
+        !listEqualsForTests(this.edges_, g.edges_)) {
       return false;
     }
     return true;
   }
 
+  private static boolean mapEqualsForTests(Map<String, Vertex> map1,
+      Map<String, Vertex> map2) {
+    if (map1.size() != map2.size()) return false;
+    Iterator<Entry<String, Vertex>> i = map1.entrySet().iterator();
+    while (i.hasNext()) {
+      Entry<String, Vertex> e = i.next();
+      String key = e.getKey();
+      Vertex value = e.getValue();
+      if (value == null) {
+        if (!(map2.get(key) == null && map2.containsKey(key))) return false;
+      } else {
+        if (!value.equalsForTests(map2.get(key))) return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean setEqualsForTests(Set<Vertex> set1, Set<Vertex> set2) {
+    if (set1.size() != set2.size()) return false;
+    for (Vertex v1 : set1) {
+      boolean found = false;
+      Iterator<Vertex> i = set2.iterator();
+      while (i.hasNext()) {
+        Vertex v2 = i.next();
+        if (v1.equalsForTests(v2)) {
+          i.remove();
+          found = true;
+        }
+      }
+      if (!found) return false;
+    }
+    return set2.isEmpty();
+  }
+
+  private static boolean listEqualsForTests(List<MultiEdge> list1,
+      List<MultiEdge> list2) {
+    ListIterator<MultiEdge> i1 = list1.listIterator();
+    ListIterator<MultiEdge> i2 = list2.listIterator();
+    while (i1.hasNext() && i2.hasNext()) {
+      MultiEdge e1 = i1.next();
+      MultiEdge e2 = i2.next();
+      if (!(e1 == null ? e2 == null : e1.equalsForTests(e2))) {
+        return false;
+      }
+    }
+    return !(i1.hasNext() || i2.hasNext());
+  }
+
   public String debugString() {
     StringBuilder builder = new StringBuilder();
     for (MultiEdge edge: edges_) {
@@ -684,16 +892,15 @@ public class ColumnLineageGraph {
     return builder.toString();
   }
 
-  public void addTargetColumnLabels(Collection<String> columnLabels) {
+  public void addTargetColumnLabels(Collection<ColumnLabel> columnLabels) {
     Preconditions.checkNotNull(columnLabels);
     targetColumnLabels_.addAll(columnLabels);
   }
 
   public void addTargetColumnLabels(FeTable dstTable) {
     Preconditions.checkNotNull(dstTable);
-    String tblFullName = dstTable.getFullName();
     for (String columnName: dstTable.getColumnNames()) {
-      targetColumnLabels_.add(tblFullName + "." + columnName);
+      targetColumnLabels_.add(new ColumnLabel(columnName, dstTable.getTableName()));
     }
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
index e0ff1c9..5c064af 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
@@ -22,6 +22,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.impala.analysis.ColumnLineageGraph.ColumnLabel;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TCreateOrAlterViewParams;
@@ -175,9 +176,9 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
    */
   protected void computeLineageGraph(Analyzer analyzer) {
     ColumnLineageGraph graph = analyzer.getColumnLineageGraph();
-    List<String> colDefs = new ArrayList<>();
+    List<ColumnLabel> colDefs = new ArrayList<>();
     for (ColumnDef colDef: finalColDefs_) {
-      colDefs.add(dbName_ + "." + getTbl() + "." + colDef.getColName());
+      colDefs.add(new ColumnLabel(colDef.getColName(), new TableName(dbName_, getTbl())));
     }
     graph.addTargetColumnLabels(colDefs);
     graph.computeLineageGraph(viewDefStmt_.getResultExprs(), analyzer);
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 5f74c45..39a255d 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.authorization.AuthorizationDelta;
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index d4cc37b..087954b 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -20,11 +20,13 @@ package org.apache.impala.planner;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.impala.analysis.AnalysisContext;
 import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.ColumnLineageGraph;
+import org.apache.impala.analysis.ColumnLineageGraph.ColumnLabel;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprSubstitutionMap;
 import org.apache.impala.analysis.InsertStmt;
@@ -192,10 +194,10 @@ public class Planner {
             // the labels of columns mentioned in the column list.
             List<String> mentionedColumns = insertStmt.getMentionedColumns();
             Preconditions.checkState(!mentionedColumns.isEmpty());
-            List<String> targetColLabels = new ArrayList<>();
+            List<ColumnLabel> targetColLabels = new ArrayList<>();
             String tblFullName = targetTable.getFullName();
             for (String column: mentionedColumns) {
-              targetColLabels.add(tblFullName + "." + column);
+              targetColLabels.add(new ColumnLabel(column, targetTable.getTableName()));
             }
             graph.addTargetColumnLabels(targetColLabels);
           } else {
@@ -213,7 +215,9 @@ public class Planner {
         }
         graph.computeLineageGraph(exprs, ctx_.getRootAnalyzer());
       } else {
-        graph.addTargetColumnLabels(ctx_.getQueryStmt().getColLabels());
+        graph.addTargetColumnLabels(ctx_.getQueryStmt().getColLabels().stream()
+            .map(col -> new ColumnLabel(col))
+            .collect(Collectors.toList()));
         graph.computeLineageGraph(resultExprs, ctx_.getRootAnalyzer());
       }
       if (LOG.isTraceEnabled()) LOG.trace("lineage: " + graph.debugString());
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 e9cb876..1022e27 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -89,7 +89,6 @@ import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.catalog.TableNotFoundException;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.View;
-import org.apache.impala.catalog.events.MetastoreEvents;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.ImpalaException;
@@ -1982,7 +1981,13 @@ public class CatalogOpExecutor {
     synchronized (metastoreDdlLock_) {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().createTable(newTable);
+        // TODO (HIVE-21807): Creating a table and retrieving the table information is
+        // not atomic.
         addSummary(response, "Table has been created.");
+        long tableCreateTime = msClient.getHiveClient().getTable(
+            newTable.getDbName(), newTable.getTableName()).getCreateTime();
+        response.setTable_name(newTable.getDbName() + "." + newTable.getTableName());
+        response.setTable_create_time(tableCreateTime);
         // 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() &&
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 631f9bd..139bb74 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -122,7 +122,6 @@ public class JniFrontend {
       new TBinaryProtocol.Factory();
   private final Frontend frontend_;
 
-
   /**
    * Create a new instance of the Jni Frontend.
    */
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
index 2917b08..3adb01f 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
@@ -740,13 +740,13 @@ public class PlannerTestBase extends FrontendTestBase {
       ColumnLineageGraph outputGraph =
         ColumnLineageGraph.fromThrift(lineageGraph);
       if (expectedGraph == null || outputGraph == null ||
-          !outputGraph.equals(expectedGraph)) {
+          !outputGraph.equalsForTests(expectedGraph)) {
         StringBuilder lineageError = new StringBuilder();
         lineageError.append("section " + Section.LINEAGE + " of query:\n"
             + query + "\n");
-        lineageError.append("Output:\n");
-        lineageError.append(outputGraph.toJson() + "\n");
-        lineageError.append("Expected:\n");
+        lineageError.append("Output:");
+        lineageError.append(TestUtils.prettyPrintJson(outputGraph.toJson() + "\n"));
+        lineageError.append("\nExpected:\n");
         lineageError.append(serializedGraph + "\n");
         errorLog.append(lineageError.toString());
       }
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/lineage.test b/testdata/workloads/functional-planner/queries/PlannerTest/lineage.test
index 9e2e598..3d35c2f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/lineage.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/lineage.test
@@ -9,7 +9,7 @@ select * from (
     "queryText":"select * from (\n  select tinyint_col + int_col x from functional.alltypes\n  union all\n  select sum(bigint_col) y from (select bigint_col from functional.alltypes) v1) v2",
     "queryId":"0:0",
     "hash":"25456c60a2e874a20732f42c7af27553",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -33,17 +33,29 @@ select * from (
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -63,7 +75,7 @@ order by b.bigint_col limit 10
     "queryText":"select sum(a.tinyint_col) over (partition by a.smallint_col order by a.id),\n  count(b.string_col), b.timestamp_col\nfrom functional.alltypes a join functional.alltypessmall b on (a.id = b.id)\nwhere a.year = 2010 and b.float_col > 0\ngroup by a.tinyint_col, a.smallint_col, a.id, b.string_col, b.timestamp_col, b.bigint_col\nhaving count(a.int_col) > 10\norder by b.bigint_col limit 10",
     "queryId":"0:0",
     "hash":"e0309eeff9811f53c82657d62c1e04eb",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -133,17 +145,29 @@ order by b.bigint_col limit 10
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.smallint_col"
+            "vertexId":"functional.alltypes.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -153,7 +177,11 @@ order by b.bigint_col limit 10
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"functional.alltypessmall.string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":6,
@@ -163,32 +191,56 @@ order by b.bigint_col limit 10
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.timestamp_col"
+            "vertexId":"functional.alltypessmall.timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bigint_col"
+            "vertexId":"functional.alltypessmall.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"functional.alltypessmall.float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -200,7 +252,7 @@ create table lineage_test_tbl as select int_col, tinyint_col from functional.all
     "queryText":"create table lineage_test_tbl as select int_col, tinyint_col from functional.alltypes",
     "queryId":"0:0",
     "hash":"407f23b24758ffcb2ac445b9703f5c44",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -226,22 +278,38 @@ create table lineage_test_tbl as select int_col, tinyint_col from functional.all
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tbl.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tbl",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tbl.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tbl",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -255,7 +323,7 @@ where a.year = 2009 and b.month = 2
     "queryText":"create table lineage_test_tbl as\nselect distinct a.int_col, a.string_col from functional.alltypes a\ninner join functional.alltypessmall b on (a.id = b.id)\nwhere a.year = 2009 and b.month = 2",
     "queryId":"0:0",
     "hash":"f3101dcb046a7d34d7ee14892a6cc94e",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -294,42 +362,74 @@ where a.year = 2009 and b.month = 2
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tbl.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tbl",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tbl.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tbl",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"functional.alltypessmall.month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -343,7 +443,7 @@ select * from
     "queryText":"create table lineage_test_tbl as\nselect * from\n  (select * from\n     (select int_col from functional.alltypestiny limit 1) v1 ) v2",
     "queryId":"0:0",
     "hash":"9c04c1e9feee35ffacf14bfcd3b363a7",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -360,12 +460,20 @@ select * from
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tbl.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tbl",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.int_col"
+            "vertexId":"functional.alltypestiny.int_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         }
     ]
 }
@@ -377,7 +485,7 @@ create table lineage_test_tblm as select * from functional_hbase.alltypes limit
     "queryText":"create table lineage_test_tblm as select * from functional_hbase.alltypes limit 5",
     "queryId":"0:0",
     "hash":"a294f36bddf2adb329eac3055a76b2b5",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -502,132 +610,236 @@ create table lineage_test_tblm as select * from functional_hbase.alltypes limit
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.id"
+            "vertexId":"functional_hbase.alltypes.id",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.bigint_col"
+            "vertexId":"functional_hbase.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.bool_col"
+            "vertexId":"functional_hbase.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.date_string_col"
+            "vertexId":"functional_hbase.alltypes.date_string_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.double_col"
+            "vertexId":"functional_hbase.alltypes.double_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.float_col"
+            "vertexId":"functional_hbase.alltypes.float_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.int_col"
+            "vertexId":"functional_hbase.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.month"
+            "vertexId":"functional_hbase.alltypes.month",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":17,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.smallint_col"
+            "vertexId":"functional_hbase.alltypes.smallint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":18,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":19,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.string_col"
+            "vertexId":"functional_hbase.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":20,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":21,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.timestamp_col"
+            "vertexId":"functional_hbase.alltypes.timestamp_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":22,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":23,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.tinyint_col"
+            "vertexId":"functional_hbase.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":24,
             "vertexType":"COLUMN",
-            "vertexId":"default.lineage_test_tblm.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"default.lineage_test_tblm",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":25,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.year"
+            "vertexId":"functional_hbase.alltypes.year",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         }
     ]
 }
@@ -642,7 +854,7 @@ functional_hbase.alltypes
     "queryText":"insert into\nfunctional_hbase.alltypes\n  values (1, 1, true, \"1999-12-01\", 2.0, 1.0, 1, 12, 2, \"abs\",\n  cast(now() as timestamp), 1, 1999)",
     "queryId":"0:0",
     "hash":"b923425ce9cc2d53d36523ec83971e67",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -754,67 +966,119 @@ functional_hbase.alltypes
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypes.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"functional_hbase.alltypes",
+                "tableCreateTime":1559151687
+            }
         }
     ]
 }
@@ -827,7 +1091,7 @@ from functional.alltypes
     "queryText":"insert into table functional.alltypesnopart (id, bool_col, timestamp_col)\nselect id, bool_col, timestamp_col\nfrom functional.alltypes",
     "queryId":"0:0",
     "hash":"b7b9474fc6b97f104bd031209438ee0e",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -926,72 +1190,128 @@ from functional.alltypes
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesnopart.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypesnopart",
+                "tableCreateTime":1559151324
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.timestamp_col"
+            "vertexId":"functional.alltypes.timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -1006,7 +1326,7 @@ where year=2009 and month=05
     "queryText":"insert into table functional.alltypessmall (smallint_col, int_col)\npartition (year=2009, month=04)\nselect smallint_col, int_col\nfrom functional.alltypes\nwhere year=2009 and month=05",
     "queryId":"0:0",
     "hash":"2ed3a6c784e1c0c7fcef226d71375180",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1142,87 +1462,155 @@ where year=2009 and month=05
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.smallint_col"
+            "vertexId":"functional.alltypes.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.month"
+            "vertexId":"functional.alltypes.month",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -1237,7 +1625,7 @@ where year=2009 and month>10
     "queryText":"insert into table functional.alltypessmall (id, string_col, int_col)\npartition (year, month)\nselect id, string_col, int_col, year, month\nfrom functional_seq_snap.alltypes\nwhere year=2009 and month>10",
     "queryId":"0:0",
     "hash":"39ac95ce0632ef1ee8b474be644971f3",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1376,92 +1764,164 @@ where year=2009 and month>10
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_seq_snap.alltypes.year"
+            "vertexId":"functional_seq_snap.alltypes.year",
+            "metadata":{
+                "tableName":"functional_seq_snap.alltypes",
+                "tableCreateTime":1559151137
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_seq_snap.alltypes.month"
+            "vertexId":"functional_seq_snap.alltypes.month",
+            "metadata":{
+                "tableName":"functional_seq_snap.alltypes",
+                "tableCreateTime":1559151137
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional_seq_snap.alltypes.id"
+            "vertexId":"functional_seq_snap.alltypes.id",
+            "metadata":{
+                "tableName":"functional_seq_snap.alltypes",
+                "tableCreateTime":1559151137
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional_seq_snap.alltypes.int_col"
+            "vertexId":"functional_seq_snap.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional_seq_snap.alltypes",
+                "tableCreateTime":1559151137
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"functional_seq_snap.alltypes.string_col"
+            "vertexId":"functional_seq_snap.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional_seq_snap.alltypes",
+                "tableCreateTime":1559151137
+            }
         },
         {
             "id":17,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -1480,7 +1940,7 @@ having min(id) > 10
     "queryText":"insert into table functional.alltypessmall\npartition (year=2009, month)\nselect min(id), min(bool_col), min(tinyint_col), min(smallint_col), min(int_col),\nmin(bigint_col), min(float_col), min(double_col), min(date_string_col), min(string_col),\nmin(timestamp_col), month\nfrom functional.alltypes\nwhere year=2009 and month>10\ngroup by month\nhaving min(id) > 10",
     "queryId":"0:0",
     "hash":"e6969c2cc67e9d6f3f985ddc6431f915",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1627,132 +2087,236 @@ having min(id) > 10
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.month"
+            "vertexId":"functional.alltypes.month",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.smallint_col"
+            "vertexId":"functional.alltypes.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.float_col"
+            "vertexId":"functional.alltypes.float_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":17,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":18,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.double_col"
+            "vertexId":"functional.alltypes.double_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":19,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":20,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.date_string_col"
+            "vertexId":"functional.alltypes.date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":21,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":22,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":23,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":24,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.timestamp_col"
+            "vertexId":"functional.alltypes.timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":25,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -1767,7 +2331,7 @@ group by int_col, tinyint_col
     "queryText":"select\nmax(tinyint_col) over(partition by int_col)\nfrom functional.alltypes\ngroup by int_col, tinyint_col",
     "queryId":"0:0",
     "hash":"83c78528e6f5325c56a3f3521b08a78d",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1798,12 +2362,20 @@ group by int_col, tinyint_col
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -1814,7 +2386,7 @@ select int_col, rank() over(order by int_col) from functional.alltypesagg
     "queryText":"select int_col, rank() over(order by int_col) from functional.alltypesagg",
     "queryId":"0:0",
     "hash":"4f1ecaaed571d2ed9f09f091f399c311",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1853,7 +2425,11 @@ select int_col, rank() over(order by int_col) from functional.alltypesagg
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.int_col"
+            "vertexId":"functional.alltypesagg.int_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":2,
@@ -1872,7 +2448,7 @@ order by a.tinyint_col, a.int_col
     "queryText":"select a.tinyint_col, a.int_col, count(a.double_col)\n  over(partition by a.tinyint_col order by a.int_col desc rows between 1 preceding and 1 following)\nfrom functional.alltypes a inner join functional.alltypessmall b on a.id = b.id\norder by a.tinyint_col, a.int_col",
     "queryId":"0:0",
     "hash":"b6e26c00b2ef17f0592ebadb0ecc21f6",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -1934,7 +2510,11 @@ order by a.tinyint_col, a.int_col
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -1944,7 +2524,11 @@ order by a.tinyint_col, a.int_col
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -1954,17 +2538,29 @@ order by a.tinyint_col, a.int_col
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.double_col"
+            "vertexId":"functional.alltypes.double_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -1983,7 +2579,7 @@ order by 2, 3, 4
     "queryText":"with v2 as\n  (select\n   double_col,\n   count(int_col) over() a,\n   sum(int_col + bigint_col) over(partition by bool_col) b\n   from\n     (select * from functional.alltypes) v1)\nselect double_col, a, b, a + b, double_col + a from v2\norder by 2, 3, 4",
     "queryId":"0:0",
     "hash":"6bf993cea0d1ab9e613674ef178916c9",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -2062,7 +2658,11 @@ order by 2, 3, 4
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.double_col"
+            "vertexId":"functional.alltypes.double_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -2072,7 +2672,11 @@ order by 2, 3, 4
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -2082,12 +2686,20 @@ order by 2, 3, 4
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
@@ -2115,7 +2727,7 @@ order by 2, 3, 4
     "queryText":"select double_col, a, b, a + b, double_col + a from\n  (select\n   double_col,\n   count(int_col) over() a,\n   sum(int_col + bigint_col) over(partition by bool_col) b\n   from\n     (select * from functional.alltypes) v1) v2\norder by 2, 3, 4",
     "queryId":"0:0",
     "hash":"811403c86e86fe630dea7bd0a6c89273",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867921,
     "edges":[
         {
@@ -2194,7 +2806,11 @@ order by 2, 3, 4
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.double_col"
+            "vertexId":"functional.alltypes.double_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -2204,7 +2820,11 @@ order by 2, 3, 4
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -2214,12 +2834,20 @@ order by 2, 3, 4
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
@@ -2249,7 +2877,7 @@ where b.month = 1
     "queryText":"select a.month, a.year, b.int_col, b.month\nfrom\n  (select year, month from functional.alltypes\n   union all\n   select year, month from functional.alltypes) a\n  inner join\n  functional.alltypessmall b\n  on (a.month = b.month)\nwhere b.month = 1",
     "queryId":"0:0",
     "hash":"e3000cd5edf2a02e1f5407810f3cc09a",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2310,7 +2938,11 @@ where b.month = 1
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.month"
+            "vertexId":"functional.alltypes.month",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -2320,7 +2952,11 @@ where b.month = 1
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -2330,12 +2966,20 @@ where b.month = 1
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.int_col"
+            "vertexId":"functional.alltypessmall.int_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"functional.alltypessmall.month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -2357,7 +3001,7 @@ where month = 1
     "queryText":"select t1.int_col, t2.month, t2.int_col + 1\nfrom (\n  select int_col, count(*)\n  from functional.alltypessmall\n  where month = 1\n  group by int_col\n  having count(*) > 1\n  order by count(*) desc limit 5\n  ) t1\njoin functional.alltypes t2 on (t1.int_col = t2.int_col)\nwhere month = 1",
     "queryId":"0:0",
     "hash":"3f1ecf7239e205342aee4979e7cb4877",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2411,7 +3055,11 @@ where month = 1
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.int_col"
+            "vertexId":"functional.alltypessmall.int_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":2,
@@ -2421,7 +3069,11 @@ where month = 1
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.month"
+            "vertexId":"functional.alltypes.month",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -2431,12 +3083,20 @@ where month = 1
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.month"
+            "vertexId":"functional.alltypessmall.month",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -2459,7 +3119,7 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
     "queryText":"select x.smallint_col, x.id, x.tinyint_col, c.id, x.int_col, x.float_col, c.string_col\nfrom functional.alltypessmall c\njoin (\n   select a.smallint_col smallint_col, a.tinyint_col tinyint_col, a.day day,\n           a.int_col int_col, a.month month, b.float_col float_col, b.id id\n   from ( select * from functional.alltypesagg a where month=1 ) a\n   join functional.alltypessmall b on (a.smallint_col = b.id)\n ) x on (x.tinyint_col = c.id)\nwhere x.day=1\nand x.int_col [...]
     "queryId":"0:0",
     "hash":"4edf165aed5982ede63f7c91074f4b44",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2556,7 +3216,11 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.smallint_col"
+            "vertexId":"functional.alltypesagg.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":2,
@@ -2566,7 +3230,11 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":4,
@@ -2576,7 +3244,11 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.tinyint_col"
+            "vertexId":"functional.alltypesagg.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":6,
@@ -2586,7 +3258,11 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.int_col"
+            "vertexId":"functional.alltypesagg.int_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":8,
@@ -2596,7 +3272,11 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"functional.alltypessmall.float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":10,
@@ -2606,17 +3286,29 @@ and x.int_col + x.float_col + cast(c.string_col as float) < 1000
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"functional.alltypessmall.string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.day"
+            "vertexId":"functional.alltypesagg.day",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.month"
+            "vertexId":"functional.alltypesagg.month",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         }
     ]
 }
@@ -2636,7 +3328,7 @@ from
     "queryText":"select c1, c2, c3\nfrom\n  (select c1, c2, c3\n   from\n     (select int_col c1, sum(float_col) c2, min(float_col) c3\n      from functional_hbase.alltypessmall\n      group by 1) x\n    order by 2,3 desc\n    limit 5\n) y",
     "queryId":"0:0",
     "hash":"8b4d1ab11721d9ebdf26666d4195eb18",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2688,7 +3380,11 @@ from
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.int_col"
+            "vertexId":"functional_hbase.alltypessmall.int_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         },
         {
             "id":2,
@@ -2698,7 +3394,11 @@ from
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.float_col"
+            "vertexId":"functional_hbase.alltypessmall.float_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         },
         {
             "id":4,
@@ -2730,7 +3430,7 @@ limit 0
     "queryText":"select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select int_col c1, tinyint_col c2, min(float_col) c3\n      from functional_hbase.alltypessmall\n      group by 1, 2\n      order by 1,2\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by 2,1 desc\nlimit 0",
     "queryId":"0:0",
     "hash":"50d3b4f249f038b0711ea75c17640fc9",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2772,7 +3472,11 @@ limit 0
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.int_col"
+            "vertexId":"functional_hbase.alltypessmall.int_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         },
         {
             "id":2,
@@ -2782,7 +3486,11 @@ limit 0
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.tinyint_col"
+            "vertexId":"functional_hbase.alltypessmall.tinyint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         }
     ]
 }
@@ -2794,7 +3502,7 @@ select int_col, string_col from functional.view_view
     "queryText":"select int_col, string_col from functional.view_view",
     "queryId":"0:0",
     "hash":"9073496459077de1332e5017977dedf5",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2825,7 +3533,11 @@ select int_col, string_col from functional.view_view
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -2835,7 +3547,11 @@ select int_col, string_col from functional.view_view
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -2847,7 +3563,7 @@ where t.id < 10
     "queryText":"select t.id from (select id from functional.alltypes_view) t\nwhere t.id < 10",
     "queryId":"0:0",
     "hash":"8ba7998033f90e1e358f4fdc7ea4251b",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2878,7 +3594,11 @@ where t.id < 10
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -2893,7 +3613,7 @@ where id in
     "queryText":"select string_col, float_col, bool_col\nfrom functional.alltypes\nwhere id in\n  (select id from functional.alltypesagg)",
     "queryId":"0:0",
     "hash":"e8ad1371d2a13e1ee9ec45689b62cdc9",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -2945,7 +3665,11 @@ where id in
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -2955,7 +3679,11 @@ where id in
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.float_col"
+            "vertexId":"functional.alltypes.float_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
@@ -2965,17 +3693,29 @@ where id in
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.id"
+            "vertexId":"functional.alltypesagg.id",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         }
     ]
 }
@@ -2993,7 +3733,7 @@ and tinyint_col < 10
     "queryText":"select 1\nfrom functional.alltypesagg a\nwhere exists\n  (select id, count(int_col) over (partition by bool_col)\n   from functional.alltypestiny b\n   where a.tinyint_col = b.tinyint_col\n   group by id, int_col, bool_col)\nand tinyint_col < 10",
     "queryId":"0:0",
     "hash":"a7500c022d29c583c31b287868a848bf",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3024,12 +3764,20 @@ and tinyint_col < 10
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.tinyint_col"
+            "vertexId":"functional.alltypesagg.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.tinyint_col"
+            "vertexId":"functional.alltypestiny.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         }
     ]
 }
@@ -3044,7 +3792,7 @@ and a.bigint_col > 10
     "queryText":"select int_col + 1, tinyint_col - 1\nfrom functional.alltypes a\nwhere a.int_col <\n  (select max(int_col) from functional.alltypesagg g where g.bool_col = true)\nand a.bigint_col > 10",
     "queryId":"0:0",
     "hash":"5e6227f323793ea4441e2a3119af2f09",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3088,7 +3836,11 @@ and a.bigint_col > 10
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -3098,22 +3850,38 @@ and a.bigint_col > 10
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.bool_col"
+            "vertexId":"functional.alltypesagg.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.int_col"
+            "vertexId":"functional.alltypesagg.int_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         }
     ]
 }
@@ -3125,7 +3893,7 @@ with t as (select int_col x, bigint_col y from functional.alltypes) select x, y
     "queryText":"with t as (select int_col x, bigint_col y from functional.alltypes) select x, y from t",
     "queryId":"0:0",
     "hash":"a7ab58d90540f28a8dfd69703632ad7a",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3156,7 +3924,11 @@ with t as (select int_col x, bigint_col y from functional.alltypes) select x, y
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
@@ -3166,7 +3938,11 @@ with t as (select int_col x, bigint_col y from functional.alltypes) select x, y
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -3179,7 +3955,7 @@ select id, int_col, string_col, year, month from t1
     "queryText":"with t1 as (select * from functional.alltypestiny)\ninsert into functional.alltypesinsert (id, int_col, string_col) partition(year, month)\nselect id, int_col, string_col, year, month from t1",
     "queryId":"0:0",
     "hash":"0bc5b3e66cc72387f74893b1f1934946",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3296,92 +4072,164 @@ select id, int_col, string_col, year, month from t1
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.year"
+            "vertexId":"year",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.year"
+            "vertexId":"functional.alltypestiny.year",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.month"
+            "vertexId":"month",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.month"
+            "vertexId":"functional.alltypestiny.month",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.id"
+            "vertexId":"functional.alltypestiny.id",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.int_col"
+            "vertexId":"functional.alltypestiny.int_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.string_col"
+            "vertexId":"functional.alltypestiny.string_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":17,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesinsert.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypesinsert",
+                "tableCreateTime":1559151331
+            }
         }
     ]
 }
@@ -3398,7 +4246,7 @@ from
     "queryText":"select lead(a) over (partition by b order by c)\nfrom\n  (select lead(id) over (partition by int_col order by bigint_col) as a,\n   max(id) over (partition by tinyint_col order by int_col) as b,\n   min(int_col) over (partition by string_col order by bool_col) as c\n   from functional.alltypes) v",
     "queryId":"0:0",
     "hash":"aa95e5e6f39fc80bb3c318a2515dc77d",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3434,32 +4282,56 @@ from
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bigint_col"
+            "vertexId":"functional.alltypes.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.bool_col"
+            "vertexId":"functional.alltypes.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -3471,7 +4343,7 @@ create view test_view_lineage as select id from functional.alltypestiny
     "queryText":"create view test_view_lineage as select id from functional.alltypestiny",
     "queryId":"0:0",
     "hash":"ff6b1ecb265afe4f03355a07238cfe37",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3488,12 +4360,20 @@ create view test_view_lineage as select id from functional.alltypestiny
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.id"
+            "vertexId":"functional.alltypestiny.id",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         }
     ]
 }
@@ -3521,7 +4401,7 @@ limit 0
     "queryText":"create view test_view_lineage (a, b) as select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select int_col c1, tinyint_col c2, min(float_col) c3\n      from functional_hbase.alltypessmall\n      group by 1, 2\n      order by 1,2\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by 2,1 desc\nlimit 0",
     "queryId":"0:0",
     "hash":"b96adf892b897da1e562c5be98724fb5",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3558,22 +4438,38 @@ limit 0
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a"
+            "vertexId":"a",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.int_col"
+            "vertexId":"functional_hbase.alltypessmall.int_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.b"
+            "vertexId":"b",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_hbase.alltypessmall.tinyint_col"
+            "vertexId":"functional_hbase.alltypessmall.tinyint_col",
+            "metadata":{
+                "tableName":"functional_hbase.alltypessmall",
+                "tableCreateTime":1559151688
+            }
         }
     ]
 }
@@ -3598,7 +4494,7 @@ create view test_view_lineage (a1, a2, a3, a4, a5, a6, a7) as
     "queryText":"create view test_view_lineage (a1, a2, a3, a4, a5, a6, a7) as\n  select x.smallint_col, x.id, x.tinyint_col, c.id, x.int_col, x.float_col, c.string_col\n  from functional.alltypessmall c\n  join (\n     select a.smallint_col smallint_col, a.tinyint_col tinyint_col, a.day day,\n           a.int_col int_col, a.month month, b.float_col float_col, b.id id\n     from ( select * from functional.alltypesagg a where month=1 ) a\n     join functional.alltypessmall b on (a.smallin [...]
     "queryId":"0:0",
     "hash":"ffbe643df8f26e92907fb45de1aeda36",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3691,77 +4587,137 @@ create view test_view_lineage (a1, a2, a3, a4, a5, a6, a7) as
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a1"
+            "vertexId":"a1",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.smallint_col"
+            "vertexId":"functional.alltypesagg.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a2"
+            "vertexId":"a2",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a3"
+            "vertexId":"a3",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.tinyint_col"
+            "vertexId":"functional.alltypesagg.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a4"
+            "vertexId":"a4",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a5"
+            "vertexId":"a5",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.int_col"
+            "vertexId":"functional.alltypesagg.int_col",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a6"
+            "vertexId":"a6",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"functional.alltypessmall.float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.a7"
+            "vertexId":"a7",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"functional.alltypessmall.string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.day"
+            "vertexId":"functional.alltypesagg.day",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypesagg.month"
+            "vertexId":"functional.alltypesagg.month",
+            "metadata":{
+                "tableName":"functional.alltypesagg",
+                "tableCreateTime":1559151363
+            }
         }
     ]
 }
@@ -3781,7 +4737,7 @@ create view test_view_lineage as
     "queryText":"create view test_view_lineage as\n  select * from (\n    select sum(a.tinyint_col) over (partition by a.smallint_col order by a.id),\n      count(b.string_col), b.timestamp_col\n    from functional.alltypes a join functional.alltypessmall b on (a.id = b.id)\n    where a.year = 2010 and b.float_col > 0\n    group by a.tinyint_col, a.smallint_col, a.id, b.string_col, b.timestamp_col, b.bigint_col\n    having count(a.int_col) > 10\n    order by b.bigint_col limit 10) t",
     "queryId":"0:0",
     "hash":"d4b9e2d63548088f911816b2ae29d7c2",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3846,67 +4802,119 @@ create view test_view_lineage as
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage._c0"
+            "vertexId":"_c0",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.tinyint_col"
+            "vertexId":"functional.alltypes.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.smallint_col"
+            "vertexId":"functional.alltypes.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage._c1"
+            "vertexId":"_c1",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.string_col"
+            "vertexId":"functional.alltypessmall.string_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.timestamp_col"
+            "vertexId":"timestamp_col",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.timestamp_col"
+            "vertexId":"functional.alltypessmall.timestamp_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.year"
+            "vertexId":"functional.alltypes.year",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.bigint_col"
+            "vertexId":"functional.alltypessmall.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.float_col"
+            "vertexId":"functional.alltypessmall.float_col",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypessmall.id"
+            "vertexId":"functional.alltypessmall.id",
+            "metadata":{
+                "tableName":"functional.alltypessmall",
+                "tableCreateTime":1559151325
+            }
         }
     ]
 }
@@ -3918,7 +4926,7 @@ alter view functional.alltypes_view as select id from functional.alltypestiny
     "queryText":"alter view functional.alltypes_view as select id from functional.alltypestiny",
     "queryId":"0:0",
     "hash":"8c9367afc562a4c04d2d40e1276646c2",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3935,12 +4943,20 @@ alter view functional.alltypes_view as select id from functional.alltypestiny
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes_view.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional.alltypes_view",
+                "tableCreateTime":1559151386
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.id"
+            "vertexId":"functional.alltypestiny.id",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         }
     ]
 }
@@ -3959,7 +4975,7 @@ select * from (
     "queryText":"select * from (\n  select int_struct_col.f1 + int_struct_col.f2 x from functional.allcomplextypes\n  where year = 2000\n  order by nested_struct_col.f2.f12.f21 limit 10\n  union all\n  select sum(f1) y from\n    (select complex_struct_col.f1 f1 from functional.allcomplextypes\n     group by 1) v1) v2",
     "queryId":"0:0",
     "hash":"4fb3ceddbf596097335af607d528f5a7",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -3993,27 +5009,47 @@ select * from (
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.complex_struct_col.f1"
+            "vertexId":"functional.allcomplextypes.complex_struct_col.f1",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_struct_col.f1"
+            "vertexId":"functional.allcomplextypes.int_struct_col.f1",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_struct_col.f2"
+            "vertexId":"functional.allcomplextypes.int_struct_col.f2",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.nested_struct_col.f2.f12.f21"
+            "vertexId":"functional.allcomplextypes.nested_struct_col.f2.f12.f21",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.year"
+            "vertexId":"functional.allcomplextypes.year",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         }
     ]
 }
@@ -4027,7 +5063,7 @@ select * from functional.allcomplextypes.int_array_col a inner join
     "queryText":"select * from functional.allcomplextypes.int_array_col a inner join\n  functional.allcomplextypes.struct_map_col m on (a.item = m.f1)",
     "queryId":"0:0",
     "hash":"8c0c64f8a4c08b82ad343ab439101957",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4089,7 +5125,11 @@ select * from functional.allcomplextypes.int_array_col a inner join
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_array_col.item"
+            "vertexId":"functional.allcomplextypes.int_array_col.item",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":2,
@@ -4099,7 +5139,11 @@ select * from functional.allcomplextypes.int_array_col a inner join
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.key"
+            "vertexId":"functional.allcomplextypes.struct_map_col.key",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":4,
@@ -4109,7 +5153,11 @@ select * from functional.allcomplextypes.int_array_col a inner join
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1"
+            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":6,
@@ -4119,7 +5167,11 @@ select * from functional.allcomplextypes.int_array_col a inner join
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.value.f2"
+            "vertexId":"functional.allcomplextypes.struct_map_col.value.f2",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         }
     ]
 }
@@ -4133,7 +5185,7 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
     "queryText":"select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col m\n  where a.item = m.f1",
     "queryId":"0:0",
     "hash":"1b0db371b32e90d33629ed7779332cf7",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4227,7 +5279,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.id"
+            "vertexId":"functional.allcomplextypes.id",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":2,
@@ -4237,7 +5293,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.year"
+            "vertexId":"functional.allcomplextypes.year",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":4,
@@ -4247,7 +5307,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.month"
+            "vertexId":"functional.allcomplextypes.month",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":6,
@@ -4257,7 +5321,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_array_col.item"
+            "vertexId":"functional.allcomplextypes.int_array_col.item",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":8,
@@ -4267,7 +5335,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.key"
+            "vertexId":"functional.allcomplextypes.struct_map_col.key",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":10,
@@ -4277,7 +5349,11 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1"
+            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":12,
@@ -4287,17 +5363,29 @@ select * from functional.allcomplextypes t, t.int_array_col a, t.struct_map_col
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.value.f2"
+            "vertexId":"functional.allcomplextypes.struct_map_col.value.f2",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_array_col"
+            "vertexId":"functional.allcomplextypes.int_array_col",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col"
+            "vertexId":"functional.allcomplextypes.struct_map_col",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         }
     ]
 }
@@ -4318,7 +5406,7 @@ select a + b as ab, c, d, e from functional.allcomplextypes t,
     "queryText":"select a + b as ab, c, d, e from functional.allcomplextypes t,\n  (select sum(item) a from t.int_array_col\n   where item < 10) v1,\n  (select count(f1) b from t.struct_map_col\n   group by key) v2,\n  (select avg(value) over(partition by key) c from t.map_map_col.value) v3,\n  (select item d from t.int_array_col\n   union all\n   select value from t.int_map_col) v4,\n  (select f21 e from t.complex_nested_struct_col.f2.f12 order by key limit 10) v5",
     "queryId":"0:0",
     "hash":"4affc0d1e384475d1ff2fc2e19643064",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4392,12 +5480,20 @@ select a + b as ab, c, d, e from functional.allcomplextypes t,
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_array_col.item"
+            "vertexId":"functional.allcomplextypes.int_array_col.item",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1"
+            "vertexId":"functional.allcomplextypes.struct_map_col.value.f1",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":3,
@@ -4407,12 +5503,20 @@ select a + b as ab, c, d, e from functional.allcomplextypes t,
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.map_map_col.value.value"
+            "vertexId":"functional.allcomplextypes.map_map_col.value.value",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.map_map_col.value.key"
+            "vertexId":"functional.allcomplextypes.map_map_col.value.key",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":6,
@@ -4422,7 +5526,11 @@ select a + b as ab, c, d, e from functional.allcomplextypes t,
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.int_map_col.value"
+            "vertexId":"functional.allcomplextypes.int_map_col.value",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":8,
@@ -4432,17 +5540,29 @@ select a + b as ab, c, d, e from functional.allcomplextypes t,
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.complex_nested_struct_col.f2.item.f12.value.f21"
+            "vertexId":"functional.allcomplextypes.complex_nested_struct_col.f2.item.f12.value.f21",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.complex_nested_struct_col.f2.item.f12.key"
+            "vertexId":"functional.allcomplextypes.complex_nested_struct_col.f2.item.f12.key",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.allcomplextypes.struct_map_col.key"
+            "vertexId":"functional.allcomplextypes.struct_map_col.key",
+            "metadata":{
+                "tableName":"functional.allcomplextypes",
+                "tableCreateTime":1559151373
+            }
         }
     ]
 }
@@ -4456,7 +5576,7 @@ where not exists (select 1 from functional.alltypes a where v.id = a.id)
     "queryText":"create view test_view_lineage as\nselect id from functional.alltypes_view v\nwhere not exists (select 1 from functional.alltypes a where v.id = a.id)",
     "queryId":"0:0",
     "hash":"e79b8abc8a682d9e0f6b2c30a6c885f3",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4482,12 +5602,20 @@ where not exists (select 1 from functional.alltypes a where v.id = a.id)
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.test_view_lineage.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"default.test_view_lineage",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -4500,7 +5628,7 @@ where k.int_col < 10
     "queryText":"select count(*) from functional_kudu.alltypes k join functional.alltypes h on k.id = h.id\nwhere k.int_col < 10",
     "queryId":"0:0",
     "hash":"7b7c92d488186d869bb6b78c97666f41",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4532,17 +5660,29 @@ where k.int_col < 10
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.alltypes.id"
+            "vertexId":"functional_kudu.alltypes.id",
+            "metadata":{
+                "tableName":"functional_kudu.alltypes",
+                "tableCreateTime":1559150985
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.alltypes.int_col"
+            "vertexId":"functional_kudu.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional_kudu.alltypes",
+                "tableCreateTime":1559150985
+            }
         }
     ]
 }
@@ -4555,7 +5695,7 @@ functional.alltypes a where a.id < 100
     "queryText":"insert into functional_kudu.testtbl select id, string_col as name, int_col as zip from\nfunctional.alltypes a where a.id < 100",
     "queryId":"0:0",
     "hash":"87a59bac56c6ad27f7af6e71af46d552",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4601,32 +5741,56 @@ functional.alltypes a where a.id < 100
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.name"
+            "vertexId":"name",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.zip"
+            "vertexId":"zip",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.int_col"
+            "vertexId":"functional.alltypes.int_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -4639,7 +5803,7 @@ functional.alltypes where id < 10
     "queryText":"insert into functional_kudu.testtbl (name, id) select string_col as name, id from\nfunctional.alltypes where id < 10",
     "queryId":"0:0",
     "hash":"0bccfdbf4118e6d5a3d94062ecb5130a",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4675,22 +5839,38 @@ functional.alltypes where id < 10
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.name"
+            "vertexId":"name",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -4703,7 +5883,7 @@ functional.alltypes where id < 10
     "queryText":"upsert into functional_kudu.testtbl (name, id) select string_col as name, id from\nfunctional.alltypes where id < 10",
     "queryId":"0:0",
     "hash":"f4c1e7b016e75012f7268f2f42ae5630",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4739,22 +5919,38 @@ functional.alltypes where id < 10
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.id"
+            "vertexId":"functional.alltypes.id",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"functional_kudu.testtbl.name"
+            "vertexId":"name",
+            "metadata":{
+                "tableName":"functional_kudu.testtbl",
+                "tableCreateTime":1559151047
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypes.string_col"
+            "vertexId":"functional.alltypes.string_col",
+            "metadata":{
+                "tableName":"functional.alltypes",
+                "tableCreateTime":1559151317
+            }
         }
     ]
 }
@@ -4769,7 +5965,7 @@ from functional.alltypestiny
     "queryText":"create table kudu_ctas primary key (id) partition by hash (id) partitions 3\nstored as kudu as select id, bool_col, tinyint_col, smallint_col, int_col,\nbigint_col, float_col, double_col, date_string_col, string_col\nfrom functional.alltypestiny",
     "queryId":"0:0",
     "hash":"de98b09af6b6ab0f0678c5fc0c4369b4",
-    "user":"progers",
+    "user":"dummy_user",
     "timestamp":1547867922,
     "edges":[
         {
@@ -4867,103 +6063,183 @@ from functional.alltypestiny
         {
             "id":0,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.id"
+            "vertexId":"id",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":1,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.id"
+            "vertexId":"functional.alltypestiny.id",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":2,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.bool_col"
+            "vertexId":"bool_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":3,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.bool_col"
+            "vertexId":"functional.alltypestiny.bool_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":4,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.tinyint_col"
+            "vertexId":"tinyint_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":5,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.tinyint_col"
+            "vertexId":"functional.alltypestiny.tinyint_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":6,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.smallint_col"
+            "vertexId":"smallint_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":7,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.smallint_col"
+            "vertexId":"functional.alltypestiny.smallint_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":8,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.int_col"
+            "vertexId":"int_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":9,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.int_col"
+            "vertexId":"functional.alltypestiny.int_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":10,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.bigint_col"
+            "vertexId":"bigint_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":11,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.bigint_col"
+            "vertexId":"functional.alltypestiny.bigint_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":12,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.float_col"
+            "vertexId":"float_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":13,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.float_col"
+            "vertexId":"functional.alltypestiny.float_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":14,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.double_col"
+            "vertexId":"double_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":15,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.double_col"
+            "vertexId":"functional.alltypestiny.double_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":16,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.date_string_col"
+            "vertexId":"date_string_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":17,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.date_string_col"
+            "vertexId":"functional.alltypestiny.date_string_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         },
         {
             "id":18,
             "vertexType":"COLUMN",
-            "vertexId":"default.kudu_ctas.string_col"
+            "vertexId":"string_col",
+            "metadata":{
+                "tableName":"default.kudu_ctas",
+                "tableCreateTime":-1
+            }
         },
         {
             "id":19,
             "vertexType":"COLUMN",
-            "vertexId":"functional.alltypestiny.string_col"
+            "vertexId":"functional.alltypestiny.string_col",
+            "metadata":{
+                "tableName":"functional.alltypestiny",
+                "tableCreateTime":1559151330
+            }
         }
     ]
 }
-====
+====
\ No newline at end of file
diff --git a/tests/custom_cluster/test_lineage.py b/tests/custom_cluster/test_lineage.py
index 240f064..35469aa 100644
--- a/tests/custom_cluster/test_lineage.py
+++ b/tests/custom_cluster/test_lineage.py
@@ -32,14 +32,10 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 
 LOG = logging.getLogger(__name__)
 
-class TestLineage(CustomClusterTestSuite):
 
+class TestLineage(CustomClusterTestSuite):
   lineage_log_dir = tempfile.mkdtemp()
 
-  query = """
-      select count(*) from functional.alltypes
-      """
-
   @classmethod
   def setup_class(cls):
     super(TestLineage, cls).setup_class()
@@ -55,7 +51,8 @@ class TestLineage(CustomClusterTestSuite):
        UNIX times."""
     LOG.info("lineage_event_log_dir is " + self.lineage_log_dir)
     before_time = int(time.time())
-    result = self.execute_query_expect_success(self.client, self.query)
+    query = "select count(*) from functional.alltypes"
+    result = self.execute_query_expect_success(self.client, query)
     profile_query_id = re.search("Query \(id=(.*)\):", result.runtime_profile).group(1)
     after_time = int(time.time())
     LOG.info("before_time " + str(before_time) + " after_time " + str(after_time))
@@ -78,3 +75,32 @@ class TestLineage(CustomClusterTestSuite):
           assert end_time <= after_time
       else:
         LOG.info("empty file: " + log_path)
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args("--lineage_event_log_dir={0}".format(lineage_log_dir))
+  def test_create_table_timestamp(self, vector, unique_database):
+    """Test that 'createTableTime' in the lineage graph are populated with valid value
+       from HMS."""
+    query = "create table {0}.lineage_test_tbl as select int_col, tinyint_col " \
+            "from functional.alltypes".format(unique_database)
+    result = self.execute_query_expect_success(self.client, query)
+    profile_query_id = re.search("Query \(id=(.*)\):", result.runtime_profile).group(1)
+
+    # Wait to flush the lineage log files.
+    time.sleep(3)
+
+    for log_filename in os.listdir(self.lineage_log_dir):
+      log_path = os.path.join(self.lineage_log_dir, log_filename)
+      # Only the coordinator's log file will be populated.
+      if os.path.getsize(log_path) > 0:
+        with open(log_path) as log_file:
+          lineage_json = json.load(log_file)
+          assert lineage_json["queryId"] == profile_query_id
+          vertices = lineage_json["vertices"]
+          for vertex in vertices:
+            if vertex["vertexId"] == "int_col":
+              assert "metadata" in vertex
+              table_name = vertex["metadata"]["tableName"]
+              table_create_time = int(vertex["metadata"]["tableCreateTime"])
+              assert "{0}.lineage_test_tbl".format(unique_database) == table_name
+              assert table_create_time != -1


[impala] 02/06: IMPALA-8562: Data cache should skip insertion of uncacheable ScanRanges

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e573b5502d4a93623dd1375e8e8febf9647c98db
Author: Michael Ho <kw...@cloudera.com>
AuthorDate: Thu May 16 17:40:17 2019 -0700

    IMPALA-8562: Data cache should skip insertion of uncacheable ScanRanges
    
    As shown in IMPALA-8561, there are some paths in the code which
    create uncacheable ScanRanges. These uncacheable ScanRanges have
    mtime of -1. 'mtime' is used for differentiating versions of files
    with the same names. An mtime == -1 means the cache entry could
    potentially be from any versions of a file with the same name.
    
    This change skips lookup or insertion of ScanRange with negative
    mtime, file offset or buffer length.
    
    Testing done: Added targeted test cases in data-cache-test
    
    Change-Id: I2294833b075a2ddcae956d9fdb04f3e85adb0391
    Reviewed-on: http://gerrit.cloudera.org:8080/13369
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/io/data-cache-test.cc | 18 ++++++++++++++++++
 be/src/runtime/io/data-cache.cc      | 14 ++++++++++++++
 be/src/runtime/io/data-cache.h       | 24 +++++++++++++-----------
 be/src/runtime/io/request-ranges.h   |  2 ++
 4 files changed, 47 insertions(+), 11 deletions(-)

diff --git a/be/src/runtime/io/data-cache-test.cc b/be/src/runtime/io/data-cache-test.cc
index 9b13918..15d995c 100644
--- a/be/src/runtime/io/data-cache-test.cc
+++ b/be/src/runtime/io/data-cache-test.cc
@@ -22,6 +22,7 @@
 
 #include "gutil/strings/join.h"
 #include "runtime/io/data-cache.h"
+#include "runtime/io/request-ranges.h"
 #include "runtime/test-env.h"
 #include "service/fe-support.h"
 #include "testutil/gtest-util.h"
@@ -262,6 +263,23 @@ TEST_F(DataCacheTest, TestBasics) {
 
   // Check that an insertion larger than the cache size will fail.
   ASSERT_FALSE(cache.Store(FNAME, MTIME, 0, test_buffer(), cache_size * 2));
+
+  // Test with uncacheable 'mtime' to make sure the entry is not stored.
+  ASSERT_FALSE(cache.Store(FNAME, BufferOpts::NEVER_CACHE, 0, test_buffer(),
+      TEMP_BUFFER_SIZE));
+  ASSERT_EQ(0, cache.Lookup(FNAME, BufferOpts::NEVER_CACHE, 0, TEMP_BUFFER_SIZE, buffer));
+
+  // Test with bad 'mtime' to make sure the entry is not stored.
+  ASSERT_FALSE(cache.Store(FNAME, -1000, 0, test_buffer(), TEMP_BUFFER_SIZE));
+  ASSERT_EQ(0, cache.Lookup(FNAME, -1000, 0, TEMP_BUFFER_SIZE, buffer));
+
+  // Test with bad 'offset' to make sure the entry is not stored.
+  ASSERT_FALSE(cache.Store(FNAME, MTIME, -2000, test_buffer(), TEMP_BUFFER_SIZE));
+  ASSERT_EQ(0, cache.Lookup(FNAME, MTIME, -2000, TEMP_BUFFER_SIZE, buffer));
+
+  // Test with bad 'buffer_len' to make sure the entry is not stored.
+  ASSERT_FALSE(cache.Store(FNAME, MTIME, 0, test_buffer(), -5000));
+  ASSERT_EQ(0, cache.Lookup(FNAME, MTIME, 0, -5000, buffer));
 }
 
 // Tests backing file rotation by setting FLAGS_data_cache_file_max_size_bytes to be 1/4
diff --git a/be/src/runtime/io/data-cache.cc b/be/src/runtime/io/data-cache.cc
index b81d30a..70b7aeb 100644
--- a/be/src/runtime/io/data-cache.cc
+++ b/be/src/runtime/io/data-cache.cc
@@ -287,6 +287,8 @@ struct DataCache::CacheKey {
  public:
   explicit CacheKey(const string& filename, int64_t mtime, int64_t offset)
     : key_(filename.size() + sizeof(mtime) + sizeof(offset)) {
+    DCHECK_GE(mtime, 0);
+    DCHECK_GE(offset, 0);
     key_.append(filename);
     key_.append(&mtime, sizeof(mtime));
     key_.append(&offset, sizeof(offset));
@@ -658,6 +660,12 @@ void DataCache::ReleaseResources() {
 int64_t DataCache::Lookup(const string& filename, int64_t mtime, int64_t offset,
     int64_t bytes_to_read, uint8_t* buffer) {
   DCHECK(!partitions_.empty());
+  // Bail out early for uncacheable ranges or invalid requests.
+  if (mtime < 0 || offset < 0 || bytes_to_read < 0) {
+    VLOG(3) << Substitute("Skipping lookup of invalid entry $0 mtime: $1 offset: $2 "
+         "bytes_to_read: $3", filename, mtime, offset, bytes_to_read);
+    return 0;
+  }
 
   // Construct a cache key. The cache key is also hashed to compute the partition index.
   const CacheKey key(filename, mtime, offset);
@@ -676,6 +684,12 @@ int64_t DataCache::Lookup(const string& filename, int64_t mtime, int64_t offset,
 bool DataCache::Store(const string& filename, int64_t mtime, int64_t offset,
     const uint8_t* buffer, int64_t buffer_len) {
   DCHECK(!partitions_.empty());
+  // Bail out early for uncacheable ranges or invalid requests.
+  if (mtime < 0 || offset < 0 || buffer_len < 0) {
+    VLOG(3) << Substitute("Skipping insertion of invalid entry $0 mtime: $1 offset: $2 "
+         "buffer_len: $3", filename, mtime, offset, buffer_len);
+    return false;
+  }
 
   // Construct a cache key. The cache key is also hashed to compute the partition index.
   const CacheKey key(filename, mtime, offset);
diff --git a/be/src/runtime/io/data-cache.h b/be/src/runtime/io/data-cache.h
index b1bf99d..6a290d9 100644
--- a/be/src/runtime/io/data-cache.h
+++ b/be/src/runtime/io/data-cache.h
@@ -45,17 +45,19 @@ class Cache;
 /// Each partition has a meta-data cache which tracks the mappings of cache keys to
 /// the locations of the cached data. A cache key is a tuple of (file's name, file's
 /// modification time, file offset) and a cache entry is a tuple of (backing file,
-/// offset in the backing file, length of the cached data, optional checksum). Each
-/// partition stores its set of cached data in backing files created on local storage.
-/// When inserting new data into the cache, the data is appended to the current backing
-/// file in use. The storage consumption of each cache entry counts towards the quota of
-/// that partition. When a partition reaches its capacity, the least recently used data
-/// in that partition is evicted. Evicted data is removed from the underlying storage by
-/// punching holes in the backing file it's stored in. As a backing file reaches a certain
-/// size (e.g. 4TB), new data will stop being appended to it and a new file will be
-/// created instead. Note that due to hole punching, the backing file is actually sparse.
-/// For instance, a backing file may look like the following after some insertion and
-/// eviction. All the holes in file consume no storage space at all.
+/// offset in the backing file, length of the cached data, optional checksum). The
+/// file's modification time is used for distinguishing between different versions of
+/// a file with a given name. Each partition stores its set of cached data in backing
+/// files created on local storage. When inserting new data into the cache, the data is
+/// appended to the current backing file in use. The storage consumption of each cache
+/// entry counts towards the quota of that partition. When a partition reaches its
+/// capacity, the least recently used data in that partition is evicted. Evicted data is
+/// removed from the underlying storage by punching holes in the backing file it's stored
+/// in. As a backing file reaches a certain size (e.g. 4TB), new data will stop being
+/// appended to it and a new file will be created instead. Note that due to hole punching,
+/// the backing file is actually sparse. For instance, a backing file may look like the
+/// following after some insertion and eviction. All the holes in file consume no storage
+/// space at all.
 ///
 /// 0                                                                             1GB
 /// +----------+----------+----------+-----------------+---------+---------+-------+
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index 29d105d..5f4053c 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -23,6 +23,7 @@
 #include <functional>
 
 #include <boost/thread/mutex.hpp>
+#include <gtest/gtest_prod.h> // for FRIEND_TEST
 
 #include "common/atomic.h"
 #include "common/hdfs.h"
@@ -198,6 +199,7 @@ struct BufferOpts {
  private:
   friend class ScanRange;
   friend class HdfsFileReader;
+  FRIEND_TEST(DataCacheTest, TestBasics);
 
   BufferOpts(
       bool try_cache, int64_t mtime, uint8_t* client_buffer, int64_t client_buffer_len)


[impala] 06/06: IMPALA-7957: Fix slot equivalences may be enforced multiple times

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ae8295118191486f31da4d8d3c9d0f7e7e5d4b3a
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Sun Apr 14 02:28:54 2019 -0700

    IMPALA-7957: Fix slot equivalences may be enforced multiple times
    
    Predicates can be divided into three types according to the way they are
    generated:
      1) origin predicates that come from the query
      2) auxiliary equal predicates generated for equivalence between a
    label(alias) and its real expression
      3) inferred predicates that inferred from the slot equivalences graph
    The slot equivalences graph (valueTransferGraph in Analyzer) is
    generated by the first two kinds of predicates. Analyzer will create
    equivalence predicates for a PlanNode based on the unassigned predicates
    and the valueTransferGraph. However, the current implementation can't
    avoid creating inferred predicates that are duplicated with previously
    created inferred predicates if they have been assigned before.
    
    Duplicated inferred predicates are either redundant or wrong. Say, if we
    create predicate p1: s1 = s2 for the current PlanNode and p1 duplicates
    with a previously inferred predicate p0: s1 = s2 (same as s2 = s1), we
    can prove that p1 is redundant or wrong:
      1) p0 must have been assigned. Otherwise, p0 will be in the unassigned
    conjuncts list and p1 won't be created.
      2) p0 must have been assigned to an offspring node of the current
    PlanNode since we create the PlanNodes in a depth first manner.
      3) The origin predicates that infer to p0 have been assigned to an
    offspring node too.
    Then, rows that should be rejected have been filtered out either by p0
    or the origin predicates that infer to p0. What's worse, assigning p1 on
    top of the origin predicates may wrongly reject rows. Hence, p1 is
    either redundant or wrong.
    
    In inferring predicates based on slot equivalence (createEquivConjuncts)
    we should update partialEquivSlots based on the previously assigned
    equivalence predicates. So slot equivalence won't be enforced multiple
    times.
    
    This patch also adds some useful TRACE level logs.
    
    Tests:
     * Add tests for UNIONs in inline-view.test
     * Run all tests locally in CORE exploration strategy
    
    Change-Id: Ida2d5d8149b217e18ebae61e136848162503653e
    Reviewed-on: http://gerrit.cloudera.org:8080/13051
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../java/org/apache/impala/analysis/Analyzer.java  |  87 ++++-
 .../apache/impala/analysis/BinaryPredicate.java    |   9 +-
 .../org/apache/impala/analysis/SlotDescriptor.java |  11 +
 .../java/org/apache/impala/analysis/SlotRef.java   |   1 +
 .../apache/impala/planner/SingleNodePlanner.java   |   7 +-
 .../queries/PlannerTest/inline-view.test           | 380 +++++++++++++++++++++
 6 files changed, 482 insertions(+), 13 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index d20c54e..45b8046 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -254,6 +254,12 @@ public class Analyzer {
     // preserve the order in which conjuncts are added.
     public final Map<ExprId, Expr> conjuncts = new LinkedHashMap<>();
 
+    // all registered inferred conjuncts (map from tuple id to conjuncts). This map is
+    // used to make sure that slot equivalences are not enforced multiple times (e.g.
+    // duplicated to previously inferred conjuncts).
+    public final Map<TupleId, List<BinaryPredicate>> assignedConjunctsByTupleId =
+        new HashMap<>();
+
     // all registered conjuncts bound by a single tuple id; used in getBoundPredicates()
     public final List<ExprId> singleTidConjuncts = new ArrayList<>();
 
@@ -1161,7 +1167,7 @@ public class Analyzer {
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("register tuple/slotConjunct: " + Integer.toString(e.getId().asInt())
-      + " " + e.toSql() + " " + e.debugString());
+          + " " + e.toSql() + " " + e.debugString());
     }
 
     if (!(e instanceof BinaryPredicate)) return;
@@ -1214,7 +1220,7 @@ public class Analyzer {
     BinaryPredicate p = new BinaryPredicate(BinaryPredicate.Operator.EQ, lhs, rhs);
     p.setIsAuxExpr();
     if (LOG.isTraceEnabled()) {
-      LOG.trace("register eq predicate: " + p.toSql() + " " + p.debugString());
+      LOG.trace("register auxiliary eq predicate: " + p.toSql() + " " + p.debugString());
     }
     registerConjunct(p);
   }
@@ -1297,6 +1303,16 @@ public class Analyzer {
     return result;
   }
 
+  public String conjunctAssignmentsDebugString() {
+    StringBuilder res = new StringBuilder();
+    for (Expr _e : globalState_.conjuncts.values()) {
+      String state = globalState_.assignedConjuncts.contains(_e.getId()) ? "assigned"
+              : "unassigned";
+      res.append("\n\t" + state + " " + _e.debugString());
+    }
+    return res.toString();
+  }
+
   /**
    * Returns true if 'e' must be evaluated after or by a join node. Note that it may
    * still be safe to evaluate 'e' elsewhere as well, but in any case 'e' must be
@@ -1658,7 +1674,10 @@ public class Analyzer {
                     != globalState_.outerJoinedTupleIds.get(destTid)));
 
           // mark all bound predicates including duplicate ones
-          if (reverseValueTransfer && !evalAfterJoin) markConjunctAssigned(srcConjunct);
+          if (reverseValueTransfer && !evalAfterJoin) {
+            markConjunctAssigned(srcConjunct);
+            if (p != srcConjunct) markConjunctAssigned(p);
+          }
         }
 
         // check if we already created this predicate
@@ -1801,6 +1820,11 @@ public class Analyzer {
   @SuppressWarnings("unchecked")
   public <T extends Expr> void createEquivConjuncts(TupleId tid, List<T> conjuncts,
       Set<SlotId> ignoreSlots) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format(
+          "createEquivConjuncts: tid=%s, conjuncts=%s, ignoreSlots=%s", tid.toString(),
+          Expr.debugString(conjuncts), ignoreSlots), new Exception("call trace"));
+    }
     // Maps from a slot id to its set of equivalent slots. Used to track equivalences
     // that have been established by 'conjuncts' and the 'ignoredsSlots'.
     DisjointSet<SlotId> partialEquivSlots = new DisjointSet<SlotId>();
@@ -1822,7 +1846,27 @@ public class Analyzer {
       // slots may not be in the same eq class due to outer joins
       if (firstEqClassId != secondEqClassId) continue;
       // update equivalences and remove redundant conjuncts
-      if (!partialEquivSlots.union(eqSlots.first, eqSlots.second)) conjunctIter.remove();
+      if (!partialEquivSlots.union(eqSlots.first, eqSlots.second)) {
+        conjunctIter.remove();
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Removed redundant conjunct: " + conjunct.debugString());
+        }
+      }
+    }
+    // For any assigned predicate, union its slots. So we can make sure that slot
+    // equivalences are not enforced multiple times.
+    if (globalState_.assignedConjunctsByTupleId.containsKey(tid)) {
+      List<BinaryPredicate> inferredConjuncts =
+          globalState_.assignedConjunctsByTupleId.get(tid);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Previously assigned predicates: " +
+            Expr.debugString(inferredConjuncts));
+      }
+      for (BinaryPredicate conjunct : inferredConjuncts) {
+        Pair<SlotId, SlotId> slots = conjunct.getEqSlots();
+        if (slots == null) continue;
+        partialEquivSlots.union(slots.first, slots.second);
+      }
     }
     // Suppose conjuncts had these predicates belonging to equivalence classes e1 and e2:
     // e1: s1 = s2, s3 = s4, s3 = s5
@@ -1892,6 +1936,9 @@ public class Analyzer {
           result.put(sccId, slotIds);
         }
         slotIds.add(slotDesc.getId());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(String.format("slot(%s) -> scc(%d)", slotDesc.getId(), sccId));
+        }
       }
     }
     return result;
@@ -2026,11 +2073,19 @@ public class Analyzer {
    * predicates.
    */
   public void computeValueTransferGraph() {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("All slots: " + SlotDescriptor.debugString(
+          globalState_.descTbl.getSlotDescs()));
+    }
     WritableGraph directValueTransferGraph =
         new WritableGraph(globalState_.descTbl.getMaxSlotId().asInt() + 1);
     constructValueTransfersFromEqPredicates(directValueTransferGraph);
     for (Pair<SlotId, SlotId> p : globalState_.registeredValueTransfers) {
       directValueTransferGraph.addEdge(p.first.asInt(), p.second.asInt());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("value transfer: from " + p.first.toString() + " to " +
+            p.second.toString());
+      }
     }
     globalState_.valueTransferGraph =
         SccCondensedGraph.condensedReflexiveTransitiveClosure(directValueTransferGraph);
@@ -2251,10 +2306,8 @@ public class Analyzer {
    * Mark predicates as assigned.
    */
   public void markConjunctsAssigned(List<Expr> conjuncts) {
-    if (conjuncts == null) return;
-    for (Expr p: conjuncts) {
-      globalState_.assignedConjuncts.add(p.getId());
-    }
+    if (conjuncts == null || conjuncts.isEmpty()) return;
+    for (Expr p: conjuncts) markConjunctAssigned(p);
   }
 
   /**
@@ -2262,6 +2315,24 @@ public class Analyzer {
    */
   public void markConjunctAssigned(Expr conjunct) {
     globalState_.assignedConjuncts.add(conjunct.getId());
+    if (Predicate.isEquivalencePredicate(conjunct)) {
+      BinaryPredicate binaryPred = (BinaryPredicate) conjunct;
+      List<TupleId> tupleIds = new ArrayList<>();
+      List<SlotId> slotIds = new ArrayList<>();
+      binaryPred.getIds(tupleIds, slotIds);
+      if (tupleIds.size() == 1 && slotIds.size() == 2
+          && binaryPred.getEqSlots() != null) {
+        // keep assigned predicates that bounds in a tuple
+        TupleId tupleId = tupleIds.get(0);
+        if (!globalState_.assignedConjunctsByTupleId.containsKey(tupleId)) {
+          globalState_.assignedConjunctsByTupleId.put(tupleId, new ArrayList<>());
+        }
+        globalState_.assignedConjunctsByTupleId.get(tupleId).add(binaryPred);
+      }
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Assigned " + conjunct.debugString());
+    }
   }
 
   public Set<ExprId> getAssignedConjuncts() {
diff --git a/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java b/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
index 2bb6625..c7f0e87 100644
--- a/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
@@ -162,10 +162,11 @@ public class BinaryPredicate extends Predicate {
 
   @Override
   public String debugString() {
-    return Objects.toStringHelper(this)
-        .add("op", op_)
-        .addValue(super.debugString())
-        .toString();
+    Objects.ToStringHelper toStrHelper = Objects.toStringHelper(this);
+    toStrHelper.add("op", op_).addValue(super.debugString());
+    if (isAuxExpr()) toStrHelper.add("isAux", true);
+    if (isInferred_) toStrHelper.add("isInferred", true);
+    return toStrHelper.toString();
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java b/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
index 5be2303..4495f19 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
@@ -18,6 +18,7 @@
 package org.apache.impala.analysis;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
@@ -290,12 +291,22 @@ public class SlotDescriptor {
     return result;
   }
 
+  public static String debugString(Collection<SlotDescriptor> slots) {
+    if (slots == null || slots.isEmpty()) return "";
+    List<String> strings = new ArrayList<>();
+    for (SlotDescriptor slot: slots) {
+      strings.add(slot.debugString());
+    }
+    return Joiner.on("\n").join(strings);
+  }
+
   public String debugString() {
     String pathStr = (path_ == null) ? "null" : path_.toString();
     String typeStr = (type_ == null ? "null" : type_.toString());
     return Objects.toStringHelper(this)
         .add("id", id_.asInt())
         .add("path", pathStr)
+        .add("label", label_)
         .add("type", typeStr)
         .add("materialized", isMaterialized_)
         .add("byteSize", byteSize_)
diff --git a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
index b329b51..7bd3b4d 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
@@ -169,6 +169,7 @@ public class SlotRef extends Expr {
   @Override
   public String debugString() {
     Objects.ToStringHelper toStrHelper = Objects.toStringHelper(this);
+    if (label_ != null) toStrHelper.add("label", label_);
     if (rawPath_ != null) toStrHelper.add("path", Joiner.on('.').join(rawPath_));
     toStrHelper.add("type", type_.toSql());
     String idStr = (desc_ == null ? "null" : Integer.toString(desc_.getId().asInt()));
diff --git a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
index 0bbc2b3..9413e11 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
@@ -355,9 +355,14 @@ public class SingleNodePlanner {
     // No point in adding SelectNode on top of an EmptyNode.
     if (root instanceof EmptySetNode) return root;
     Preconditions.checkNotNull(root);
-    // Gather unassigned conjuncts and generate predicates to enfore
+    // Gather unassigned conjuncts and generate predicates to enforce
     // slot equivalences for each tuple id.
     List<Expr> conjuncts = analyzer.getUnassignedConjuncts(root);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("unassigned conjuncts for (Node %s): %s",
+          root.getDisplayLabel(), Expr.debugString(conjuncts)));
+      LOG.trace("all conjuncts: " + analyzer.conjunctAssignmentsDebugString());
+    }
     for (TupleId tid: tupleIds) {
       analyzer.createEquivConjuncts(tid, conjuncts);
     }
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
index e2b7d98..872821d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
@@ -1840,3 +1840,383 @@ PLAN-ROOT SINK
    HDFS partitions=4/4 files=4 size=6.32KB
    row-size=8B cardinality=100
 ====
+# IMPALA-7957: Slot equivalences should not be enforced multiple times.
+# Without this patch, the planner will incorrectly generated a SELECT node with a wrong
+# predicate "functional.alltypestiny.id = functional.alltypestiny.int_col" on top of the
+# JOIN node. So LEFT JOIN results with NULL values will be incorrectly rejects.
+SELECT t.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+   FROM functional.alltypestiny
+   WHERE int_col = id) t2
+ON (t.id = t2.id)
+UNION ALL
+VALUES (NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=4B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  row-size=12B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# IMPALA-7957: Slot equivalences should not be enforced multiple times.
+# Coverage for UNION DISTINCT
+SELECT t.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+  FROM functional.alltypestiny
+  WHERE int_col = id) t2
+ON (t.id = t2.id)
+UNION DISTINCT
+VALUES (NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+04:AGGREGATE [FINALIZE]
+|  group by: id
+|  row-size=4B cardinality=9
+|
+00:UNION
+|  constant-operands=1
+|  row-size=4B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  row-size=12B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# IMPALA-7957: Slot equivalences should not be enforced multiple times.
+# The WHERE predicate "t.int_col = t.id" is pushed down to the scan node of t.
+SELECT t.id, t.int_col
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+  FROM functional.alltypestiny) t2
+ON (t.id = t2.id)
+WHERE t.int_col = t.id
+UNION ALL
+VALUES (NULL, NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=1
+|
+03:HASH JOIN [RIGHT OUTER JOIN]
+|  hash predicates: id = t.id
+|  runtime filters: RF000 <- t.id
+|  row-size=12B cardinality=1
+|
+|--01:SCAN HDFS [functional.alltypestiny t]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: t.int_col = t.id
+|     row-size=8B cardinality=1
+|
+02:SCAN HDFS [functional.alltypestiny]
+   HDFS partitions=4/4 files=4 size=460B
+   runtime filters: RF000 -> id
+   row-size=4B cardinality=8
+====
+# Regression test for IMPALA-7957. Comparing to the test above, this test changes the
+# WHERE clause to target on the rhs of the LEFT JOIN. The WHERE predicate is correctly
+# duplicated and pushed down to the SCAN node.
+SELECT t2.id, t2.int_col
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+  FROM functional.alltypestiny) t2
+ON (t.id = t2.id)
+WHERE t2.int_col = t2.id
+UNION ALL
+VALUES (NULL, NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  other predicates: int_col = id
+|  row-size=12B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: functional.alltypestiny.int_col = functional.alltypestiny.id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# Regression test for IMPALA-7957. The inline view t2 has a ORDER BY LIMIT clause.
+# No behavior changes after the patch.
+SELECT t2.id, t2.int_col
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+  FROM functional.alltypestiny
+  ORDER BY id LIMIT 2) t2
+ON (t.id = t2.id)
+WHERE t2.int_col = t2.id
+UNION ALL
+VALUES (NULL, NULL);
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=9
+|
+05:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  other predicates: int_col = id
+|  row-size=12B cardinality=8
+|
+|--04:SELECT
+|  |  predicates: id = int_col
+|  |  row-size=8B cardinality=0
+|  |
+|  03:TOP-N [LIMIT=2]
+|  |  order by: id ASC
+|  |  row-size=8B cardinality=2
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     row-size=8B cardinality=8
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# Regression test for IMPALA-7957. The inline view t2 has an analytic function.
+# No behavior changes after the patch.
+SELECT t2.id, t2.int_col
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col, count(int_col) over (partition by int_col) int_sum
+  FROM functional.alltypestiny
+  ORDER BY id LIMIT 2) t2
+ON (t.id = t2.id)
+WHERE t2.int_col = t2.id
+UNION ALL
+VALUES (NULL, NULL);
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=9
+|
+05:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  other predicates: int_col = id
+|  row-size=12B cardinality=8
+|
+|--04:SELECT
+|  |  predicates: id = int_col
+|  |  row-size=8B cardinality=0
+|  |
+|  03:TOP-N [LIMIT=2]
+|  |  order by: id ASC
+|  |  row-size=8B cardinality=2
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     row-size=8B cardinality=8
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# Regression test for IMPALA-7957. The first union operand has an ORDER BY LIMIT clause.
+SELECT t.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+   FROM functional.alltypestiny
+   WHERE int_col = id) t2
+ON (t.id = t2.id)
+ORDER BY 1 LIMIT 3
+UNION ALL
+VALUES (NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  pass-through-operands: all
+|  row-size=4B cardinality=4
+|
+04:TOP-N [LIMIT=3]
+|  order by: id ASC
+|  row-size=4B cardinality=3
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  row-size=12B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# Regression test for IMPALA-7957. The first union operand has a GROUP BY clause.
+SELECT t.id, sum(t.int_col)
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+   FROM functional.alltypestiny
+   WHERE int_col = id) t2
+ON (t.id = t2.id)
+GROUP BY 1
+UNION ALL
+VALUES (NULL, NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  pass-through-operands: all
+|  row-size=12B cardinality=9
+|
+04:AGGREGATE [FINALIZE]
+|  output: sum(t.int_col)
+|  group by: t.id
+|  row-size=12B cardinality=8
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  row-size=16B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=8B cardinality=8
+====
+# IMPALA-7957: Add the same predicate *outside* the left join, it should still be enforced.
+SELECT t.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col
+  FROM functional.alltypestiny
+  WHERE int_col = id) t2
+ON (t.id = t2.id) where t2.id = t2.int_col
+UNION ALL
+VALUES (NULL);
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=4B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  other predicates: id = int_col
+|  row-size=12B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id
+|     row-size=8B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# IMPALA-7957: Same thing except with a predicate on a different column (i.e. adding it
+# to the equivalence class)
+SELECT t.id, t2.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col, smallint_col
+  FROM functional.alltypestiny
+  WHERE int_col = id) t2
+ON (t.id = t2.id) where t2.int_col = t2.smallint_col
+UNION ALL
+VALUES (NULL, NULL)
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  other predicates: int_col = smallint_col
+|  row-size=14B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id, id = smallint_col
+|     row-size=10B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====
+# IMPALA-7957: Multiple predicates that must not be placed above the join
+SELECT t.id, t2.id
+FROM functional.alltypestiny t
+LEFT JOIN
+  (SELECT id, int_col, smallint_col
+  FROM functional.alltypestiny
+  WHERE int_col = id and smallint_col = id and tinyint_col = id) t2
+ON (t.id = t2.id)
+UNION ALL
+VALUES (NULL, NULL);
+---- PLAN
+PLAN-ROOT SINK
+|
+00:UNION
+|  constant-operands=1
+|  row-size=8B cardinality=9
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: t.id = id
+|  row-size=15B cardinality=8
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     HDFS partitions=4/4 files=4 size=460B
+|     predicates: int_col = id, smallint_col = id, tinyint_col = id
+|     row-size=11B cardinality=1
+|
+01:SCAN HDFS [functional.alltypestiny t]
+   HDFS partitions=4/4 files=4 size=460B
+   row-size=4B cardinality=8
+====


[impala] 04/06: IMPALA-7369: part 1: Implement TRUNC, DATE_TRUNC, EXTRACT, DATE_PART functions for DATE

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f0678b06e64b57e0b26ca73577444e1941925dbc
Author: Attila Jeges <at...@cloudera.com>
AuthorDate: Thu May 16 17:38:22 2019 +0200

    IMPALA-7369: part 1: Implement TRUNC, DATE_TRUNC, EXTRACT, DATE_PART functions for DATE
    
    These functions are somewhat similar in that each of them takes a DATE
    argument and a time unit to work with.
    
    They work identically to the corresponding TIMESTAMP functions. The
    only difference is that the DATE functions don't accept time-of-day
    units.
    
    TRUNC(DATE d, STRING unit)
    Truncates a DATE value to the specified time unit. The 'unit' argument
    is case insensitive. This argument string can be one of:
      SYYYY, YYYY, YEAR, SYEAR, YYY, YY, Y: Year.
      Q: Quarter.
      MONTH, MON, MM, RM: Month.
      DDD, DD, J: Day.
      DAY, DY, D: Starting day (Monday) of the week.
      WW: Truncates to the most recent date, no later than 'd', which is
          on the same day of the week as the first day of year.
      W: Truncates to the most recent date, no later than 'd', which is on
         the same day of the week as the first day of month.
    
    The impelementation mirrors Impala's TRUNC(TIMESTAMP ts, STRING unit)
    function. Hive and Oracle SQL have a similar function too.
    Reference:
    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions201.htm
    .
    
    DATE_TRUNC(STRING unit, DATE d)
    Truncates a DATE value to the specified precision. The 'unit' argument
    is case insensitive. This argument string can be one of: DAY, WEEK,
    MONTH, YEAR, DECADE, CENTURY, MILLENNIUM.
    
    The implementation mirrors Impala's DATE_TRUNC(STRING unit,
    TIMESTAMP ts) function. Vertica has a similar function too.
    Reference:
    https://my.vertica.com/docs/8.1.x/HTML/index.htm#Authoring/
        SQLReferenceManual/Functions/Date-Time/DATE_TRUNC.htm
    .
    
    EXTRACT(DATE d, STRING unit), EXTRACT(unit FROM DATE d)
    Returns one of the numeric date fields from a DATE value. The 'unit'
    string can be one of YEAR, QUARTER, MONTH, DAY. This argument value is
    case-insensitive.
    
    The implementation mirrors that Impala's EXTRACT(TIMESTAMP ts,
    STRING unit). Hive and Oracle SQL have a similar function too.
    Reference:
    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
    .
    
    DATE_PART(STRING unit, DATE date)
    Similar to EXTRACT(), with the argument order reversed. Supports the
    same date units as EXTRACT().
    
    The implementation mirrors Impala's DATE_PART(STRING unit,
    TIMESTAMP ts) function.
    
    Change-Id: I843358a45eb5faa2c134994600546fc1d0a797c8
    Reviewed-on: http://gerrit.cloudera.org:8080/13363
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/CMakeLists.txt                   |   1 +
 be/src/benchmarks/date-benchmark.cc                | 118 +++++
 be/src/exprs/expr-test.cc                          | 252 +++++++++
 be/src/exprs/udf-builtins-ir.cc                    | 295 +++++------
 be/src/exprs/udf-builtins.cc                       | 567 ++++++++++++++++++---
 be/src/exprs/udf-builtins.h                        | 160 ++++--
 be/src/runtime/date-test.cc                        |  87 ++++
 be/src/runtime/date-value.cc                       |  94 +++-
 be/src/runtime/date-value.h                        |  14 +
 common/function-registry/impala_functions.py       |  42 +-
 .../apache/impala/analysis/ExtractFromExpr.java    |  32 +-
 .../apache/impala/analysis/AnalyzeExprsTest.java   |  11 +-
 .../queries/QueryTest/hdfs-partitions.test         |   2 +-
 13 files changed, 1373 insertions(+), 302 deletions(-)

diff --git a/be/src/benchmarks/CMakeLists.txt b/be/src/benchmarks/CMakeLists.txt
index dfe3d23..83e3504 100644
--- a/be/src/benchmarks/CMakeLists.txt
+++ b/be/src/benchmarks/CMakeLists.txt
@@ -59,5 +59,6 @@ ADD_BE_BENCHMARK(string-search-benchmark)
 ADD_BE_BENCHMARK(thread-create-benchmark)
 ADD_BE_BENCHMARK(tuple-layout-benchmark)
 ADD_BE_BENCHMARK(convert-timestamp-benchmark)
+ADD_BE_BENCHMARK(date-benchmark)
 
 target_link_libraries(hash-benchmark Experiments)
diff --git a/be/src/benchmarks/date-benchmark.cc b/be/src/benchmarks/date-benchmark.cc
new file mode 100644
index 0000000..c0f176c
--- /dev/null
+++ b/be/src/benchmarks/date-benchmark.cc
@@ -0,0 +1,118 @@
+// 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 <iostream>
+#include <vector>
+
+#include "cctz/civil_time.h"
+#include "gutil/basictypes.h"
+#include "runtime/date-value.h"
+#include "util/benchmark.h"
+#include "util/cpu-info.h"
+
+#include "common/names.h"
+
+using namespace impala;
+
+// Machine Info: Intel(R) Core(TM) i5-6600 CPU @ 3.30GHz
+// ToYear:             Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+//                                                                   (relative) (relative) (relative)
+// --------------------------------------------------------------------------------------------------
+//           TestToYearMonthDay              0.167    0.169    0.169         1X         1X         1X
+//                   TestToYear               4.63     4.72     4.72      27.8X      27.8X      27.8X
+
+class TestData {
+public:
+  void AddRange(const DateValue& dv_min, const DateValue& dv_max) {
+    DCHECK(dv_min.IsValid());
+    DCHECK(dv_max.IsValid());
+    DateValue dv = dv_min;
+    while (dv < dv_max) {
+      date_.push_back(dv);
+      dv = dv.AddDays(1);
+      DCHECK(dv.IsValid());
+    }
+    to_ymd_result_year_.resize(date_.size());
+    to_y_result_year_.resize(date_.size());
+  }
+
+  void TestToYearMonthDay(int batch_size) {
+    DCHECK(date_.size() == to_ymd_result_year_.size());
+    int month, day;
+    for (int i = 0; i < batch_size; ++i) {
+      int n = date_.size();
+      for (int j = 0; j < n; ++j) {
+        ignore_result(date_[j].ToYearMonthDay(&to_ymd_result_year_[j], &month, &day));
+      }
+    }
+  }
+
+  void TestToYear(int batch_size) {
+    DCHECK(date_.size() == to_y_result_year_.size());
+    for (int i = 0; i < batch_size; ++i) {
+      int n = date_.size();
+      for (int j = 0; j < n; ++j) {
+        ignore_result(date_[j].ToYear(&to_y_result_year_[j]));
+      }
+    }
+  }
+
+  bool CheckResults() {
+    DCHECK(to_ymd_result_year_.size() == to_y_result_year_.size());
+    bool ok = true;
+    for (int i = 0; i < to_ymd_result_year_.size(); ++i) {
+      if (to_ymd_result_year_[i] != to_y_result_year_[i]) {
+        cerr << "Incorrect results " << date_[i] << ": "
+            << to_ymd_result_year_[i] << " != " << to_y_result_year_[i]
+            << endl;
+        ok = false;
+      }
+    }
+    return ok;
+  }
+
+private:
+  vector<DateValue> date_;
+  vector<int> to_ymd_result_year_;
+  vector<int> to_y_result_year_;
+};
+
+void TestToYearMonthDay(int batch_size, void* d) {
+  TestData* data = reinterpret_cast<TestData*>(d);
+  data->TestToYearMonthDay(batch_size);
+}
+
+void TestToYear(int batch_size, void* d) {
+  TestData* data = reinterpret_cast<TestData*>(d);
+  data->TestToYear(batch_size);
+}
+
+int main(int argc, char* argv[]) {
+  CpuInfo::Init();
+  cout << Benchmark::GetMachineInfo() << endl;
+
+  TestData data;
+  data.AddRange(DateValue(1965, 1, 1), DateValue(2020, 12, 31));
+
+  // Benchmark DateValue::ToYearMonthDay() vs DateValue::ToYear()
+  Benchmark suite("ToYear");
+  suite.AddBenchmark("TestToYearMonthDay", TestToYearMonthDay, &data);
+  suite.AddBenchmark("TestToYear", TestToYear, &data);
+  cout << suite.Measure();
+
+  return data.CheckResults() ? 0 : 1;
+}
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index f8486b9..f89d953 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -7421,6 +7421,258 @@ TEST_P(ExprTest, TimestampFunctions) {
   TestIsNull("unix_micros_to_utc_timestamp(253402300800000000)", TYPE_TIMESTAMP);
 }
 
+TEST_P(ExprTest, TruncForDateTest) {
+  // trunc(date, string unit)
+  // Truncate date to year
+  for (const string unit: { "SYYYY", "YYYY", "YEAR", "SYEAR", "YYY", "YY", "Y" }) {
+    const string expr = "trunc(date'2014-04-01', '" + unit + "')";
+    TestDateValue(expr, DateValue(2014, 1, 1));
+  }
+  TestDateValue("trunc(date'2000-01-01', 'Y')", DateValue(2000, 1, 1));
+
+  // Truncate date to quarter
+  TestDateValue("trunc(date'2000-01-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-02-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-03-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-04-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-05-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-06-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-07-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-08-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-09-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-10-01', 'Q')", DateValue(2000, 10, 1));
+  TestDateValue("trunc(date'2000-11-01', 'Q')", DateValue(2000, 10, 1));
+  TestDateValue("trunc(date'2000-12-01', 'Q')", DateValue(2000, 10, 1));
+
+  // Truncate date to month
+  for (const string& unit: { "MONTH", "MON", "MM", "RM" }) {
+    const string expr = "trunc(date'2001-02-05', '" + unit + "')";
+    TestDateValue(expr, DateValue(2001, 2, 1));
+  }
+  TestDateValue("trunc(date'2001-01-01', 'MM')", DateValue(2001, 1, 1));
+  TestDateValue("trunc(date'2001-12-29', 'MM')", DateValue(2001, 12, 1));
+
+  // Same day of the week as the first day of the year
+  TestDateValue("trunc(date'2014-01-07', 'WW')", DateValue(2014, 1, 1));
+  TestDateValue("trunc(date'2014-01-08', 'WW')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-09', 'WW')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-14', 'WW')", DateValue(2014, 1, 8));
+
+  // Same day of the week as the first day of the month
+  TestDateValue("trunc(date'2014-01-07', 'W')", DateValue(2014, 1, 1));
+  TestDateValue("trunc(date'2014-01-08', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-09', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-14', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-02-01', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-02', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-03', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-07', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-08', 'W')", DateValue(2014, 2, 8));
+  TestDateValue("trunc(date'2014-02-24', 'W')", DateValue(2014, 2, 22));
+
+  // Truncate to day, i.e. leave the date intact
+  for (const string& unit: { "DDD", "DD", "J" }) {
+    const string expr = "trunc(date'2014-01-08', '" + unit + "')";
+    TestDateValue(expr, DateValue(2014, 1, 8));
+  }
+
+  // Truncate date to starting day of the week
+  for (const string& unit: { "DAY", "DY", "D" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestDateValue(expr, DateValue(2012, 9, 10));
+  }
+  TestDateValue("trunc(date'2012-09-11', 'D')", DateValue(2012, 9, 10));
+  TestDateValue("trunc(date'2012-09-12', 'D')", DateValue(2012, 9, 10));
+  TestDateValue("trunc(date'2012-09-16', 'D')", DateValue(2012, 9, 10));
+
+  // Test upper limit
+  TestDateValue("trunc(date'9999-12-31', 'YYYY')", DateValue(9999, 1, 1));
+  TestDateValue("trunc(date'9999-12-31', 'Q')", DateValue(9999, 10, 1));
+  TestDateValue("trunc(date'9999-12-31', 'MONTH')", DateValue(9999, 12, 1));
+  TestDateValue("trunc(date'9999-12-31', 'W')", DateValue(9999, 12, 29));
+  TestDateValue("trunc(date'9999-12-31', 'WW')", DateValue(9999, 12, 31));
+  TestDateValue("trunc(date'9999-12-31', 'DDD')", DateValue(9999, 12, 31));
+  TestDateValue("trunc(date'9999-12-31', 'DAY')", DateValue(9999, 12, 27));
+
+  // Test lower limit
+  TestDateValue("trunc(date'0000-01-01', 'YYYY')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'Q')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-03-31', 'Q')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'MONTH')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'W')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-07', 'W')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-08', 'W')", DateValue(0, 1, 8));
+  TestDateValue("trunc(date'0000-01-01', 'WW')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-07', 'WW')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-08', 'WW')", DateValue(0, 1, 8));
+  TestDateValue("trunc(date'0000-01-04', 'DAY')", DateValue(0, 1, 3));
+  TestDateValue("trunc(date'0000-01-03', 'DAY')", DateValue(0, 1, 3));
+  TestIsNull("trunc(date'0000-01-02', 'DAY')", TYPE_DATE);
+  TestIsNull("trunc(date'0000-01-01', 'DAY')", TYPE_DATE);
+
+  // Truncating date to hour or minute returns an error
+  for (const string& unit: { "HH", "HH12", "HH24", "MI" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestNonOkStatus(expr);  // Unsupported Truncate Unit
+  }
+
+  // Invalid trunc unit
+  for (const string& unit: { "MIN", "XXYYZZ", "" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestNonOkStatus(expr);  // Invalid Truncate Unit
+  }
+
+  TestIsNull("trunc(cast(NULL as date), 'DDD')", TYPE_DATE);
+  TestNonOkStatus("trunc(cast(NULL as date), NULL)");
+}
+
+TEST_P(ExprTest, DateTruncForDateTest) {
+  TestDateValue("date_trunc('MILLENNIUM', date '2016-05-08')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '3000-12-31')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '3001-01-01')", DateValue(3001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '2016-05-08')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '2116-05-08')", DateValue(2101, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '2116-05-08')", DateValue(2110, 1, 1));
+  TestDateValue("date_trunc('YEAR', date '2016-05-08')", DateValue(2016, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '2016-05-08')", DateValue(2016, 5, 1));
+  TestDateValue("date_trunc('WEEK', date '2116-05-08')", DateValue(2116, 5, 4));
+  TestDateValue("date_trunc('WEEK', date '2017-01-01')", DateValue(2016,12,26));
+  TestDateValue("date_trunc('WEEK', date '2017-01-02')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-07')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-08')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-09')", DateValue(2017, 1, 9));
+  TestDateValue("date_trunc('DAY', date '1416-05-08')", DateValue(1416, 5, 8));
+
+  // Test upper limit
+  TestDateValue("date_trunc('MILLENNIUM', date '9999-12-31')", DateValue(9001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '9999-12-31')", DateValue(9901, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '9999-12-31')", DateValue(9990, 1, 1));
+  TestDateValue("date_trunc('YEAR', date '9999-12-31')", DateValue(9999, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '9999-12-31')", DateValue(9999, 12, 1));
+  TestDateValue("date_trunc('WEEK', date '9999-12-31')", DateValue(9999, 12, 27));
+  TestDateValue("date_trunc('DAY', date '9999-12-31')", DateValue(9999, 12, 31));
+
+  // Test lower limit for millennium
+  TestDateValue("date_trunc('MILLENNIUM', date '1001-01-01')", DateValue(1001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '1000-01-01')", DateValue(1, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '0001-01-01')", DateValue(1, 1, 1));
+  TestIsNull("date_trunc('MILLENNIUM', date '0000-01-01')", TYPE_DATE);
+
+  // Test lower limit for century
+  TestDateValue("date_trunc('CENTURY', date '0101-01-01')", DateValue(101, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '0100-01-01')", DateValue(1, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '0001-01-01')", DateValue(1, 1, 1));
+  TestIsNull("date_trunc('CENTURY', date '0000-01-01')", TYPE_DATE);
+
+  // Test lower limit for decade
+  TestDateValue("date_trunc('DECADE', date '0001-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '0000-01-01')", DateValue(0, 1, 1));
+
+  // Test lower limit for year, month, day
+  TestDateValue("date_trunc('YEAR', date '0000-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '0000-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('DAY', date '0000-01-01')", DateValue(0, 1, 1));
+
+  // Test lower limit for week
+  TestDateValue("date_trunc('WEEK', date '0000-01-09')", DateValue(0, 1, 3));
+  TestDateValue("date_trunc('WEEK', date '0000-01-03')", DateValue(0, 1, 3));
+  TestIsNull("date_trunc('WEEK', date '0000-01-02')", TYPE_DATE);
+  TestIsNull("date_trunc('WEEK', date '0000-01-01')", TYPE_DATE);
+
+  // Test invalid input.
+  // Truncating date to hour or minute returns an error
+  for (const string& unit: { "HOUR", "MINUTE", "SECOND", "MILLISECONDS",
+      "MICROSECONDS" }) {
+    const string expr = "date_trunc('" + unit + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // Unsupported Date Truncate Unit
+  }
+
+  // Invalid trunc unit
+  for (const string& unit: { "YEARR", "XXYYZZ", "" }) {
+    const string expr = "date_trunc('" + unit + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // Invalid Date Truncate Unit
+  }
+
+  TestIsNull("date_trunc('DAY', cast(NULL as date))", TYPE_DATE);
+  TestNonOkStatus("date_trunc(NULL, cast(NULL as date))");  // Invalid Date Truncate Unit
+}
+
+TEST_P(ExprTest, ExtractAndDatePartForDateTest) {
+  // extract as a regular function
+  TestValue("extract(date '2006-05-12', 'YEAR')", TYPE_BIGINT, 2006);
+  TestValue("extract(date '2006-05-12', 'quarter')", TYPE_BIGINT, 2);
+  TestValue("extract(date '2006-05-12', 'MoNTH')", TYPE_BIGINT, 5);
+  TestValue("extract(date '2006-05-12', 'DaY')", TYPE_BIGINT, 12);
+
+  // extract using FROM keyword
+  TestValue("extract(year from date '2006-05-12')", TYPE_BIGINT, 2006);
+  TestValue("extract(QUARTER from date '2006-05-12')", TYPE_BIGINT, 2);
+  TestValue("extract(mOnTh from date '2006-05-12')", TYPE_BIGINT, 5);
+  TestValue("extract(dAy from date '2006-05-12')", TYPE_BIGINT, 12);
+
+  // Test upper limit
+  TestValue("extract(date '9999-12-31', 'YEAR')", TYPE_BIGINT, 9999);
+  TestValue("extract(quarter from date '9999-12-31')", TYPE_BIGINT, 4);
+  TestValue("extract(date '9999-12-31', 'month')", TYPE_BIGINT, 12);
+  TestValue("extract(DAY from date '9999-12-31')", TYPE_BIGINT, 31);
+
+  // Test lower limit
+  TestValue("extract(date '0000-01-01', 'YEAR')", TYPE_BIGINT, 0);
+  TestValue("extract(quarter from date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("extract(date '0000-01-01', 'month')", TYPE_BIGINT, 1);
+  TestValue("extract(DAY from date '0000-01-01')", TYPE_BIGINT, 1);
+
+  // Time of day extract fields are not supported
+  for (const string& field: { "MINUTE", "SECOND", "MILLISECOND", "EPOCH" }) {
+    const string expr = "extract(date '2012-09-10', '" + field + "')";
+    TestNonOkStatus(expr);  // Unsupported Extract Field
+  }
+
+  // Invalid extract fields
+  for (const string& field: { "foo", "SSECOND", "" }) {
+    const string expr = "extract(date '2012-09-10', '" + field + "')";
+    TestNonOkStatus(expr);  // Invalid Extract Field
+  }
+
+  TestIsNull("extract(cast(NULL as date), 'YEAR')", TYPE_BIGINT);
+  TestIsNull("extract(YEAR from cast(NULL as date))", TYPE_BIGINT);
+  TestNonOkStatus("extract(cast(NULL as date), NULL)");
+
+  // date_part, same as extract function but with arguments swapped
+  TestValue("date_part('YEAR', date '2006-05-12')", TYPE_BIGINT, 2006);
+  TestValue("date_part('QuarTer', date '2006-05-12')", TYPE_BIGINT, 2);
+  TestValue("date_part('Month', date '2006-05-12')", TYPE_BIGINT, 5);
+  TestValue("date_part('Day', date '2006-05-12')", TYPE_BIGINT, 12);
+
+  // Test upper limit
+  TestValue("date_part('YEAR', date '9999-12-31')", TYPE_BIGINT, 9999);
+  TestValue("date_part('QUARTER', '9999-12-31')", TYPE_BIGINT, 4);
+  TestValue("date_part('month', date '9999-12-31')", TYPE_BIGINT, 12);
+  TestValue("date_part('DAY', date '9999-12-31')", TYPE_BIGINT, 31);
+
+  // Test lower limit
+  TestValue("date_part('year', date '0000-01-01')", TYPE_BIGINT, 0);
+  TestValue("date_part('quarter', date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("date_part('MONTH', date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("date_part('DAY', date '0000-01-01')", TYPE_BIGINT, 1);
+
+  // Time of day extract fields are not supported
+  for (const string& field: { "MINUTE", "SECOND", "MILLISECOND", "EPOCH" }) {
+    const string expr = "date_part('" + field + "', date '2012-09-10')";
+    // Unsupported Date Part Field
+    TestNonOkStatus(expr);
+  }
+
+  // Invalid extract fields
+  for (const string& field: { "foo", "SSECOND", "" }) {
+    const string expr = "date_part('" + field + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // Invalid Date Part Field
+  }
+
+  TestIsNull("date_part('YEAR', cast(NULL as date))", TYPE_BIGINT);
+  TestNonOkStatus("date_part(MULL, cast(NULL as date))");  // Invalid Date Part Field
+}
+
 TEST_P(ExprTest, ConditionalFunctions) {
   // If first param evaluates to true, should return second parameter,
   // false or NULL should return the third.
diff --git a/be/src/exprs/udf-builtins-ir.cc b/be/src/exprs/udf-builtins-ir.cc
index c870fae..05694b7 100644
--- a/be/src/exprs/udf-builtins-ir.cc
+++ b/be/src/exprs/udf-builtins-ir.cc
@@ -25,20 +25,12 @@
 #include <string>
 
 #include "gen-cpp/Exprs_types.h"
-#include "gutil/walltime.h"
 #include "runtime/runtime-state.h"
-#include "runtime/timestamp-value.h"
 #include "udf/udf-internal.h"
-#include "util/bit-util.h"
 
 #include "common/names.h"
 
-using boost::gregorian::date;
-using boost::gregorian::date_duration;
-using boost::posix_time::ptime;
-using boost::posix_time::time_duration;
 using namespace impala;
-using namespace strings;
 
 DoubleVal UdfBuiltins::Abs(FunctionContext* context, const DoubleVal& v) {
   if (v.is_null) return v;
@@ -104,161 +96,6 @@ BooleanVal UdfBuiltins::IsInf(FunctionContext* context, const DoubleVal& val) {
   return BooleanVal(std::isinf(val.val));
 }
 
-TimestampVal UdfBuiltins::Trunc(FunctionContext* context, const TimestampVal& tv,
-    const StringVal &unit_str) {
-  return TruncImpl(context, tv, unit_str);
-}
-
-TimestampVal UdfBuiltins::DateTrunc(
-    FunctionContext* context, const StringVal& unit_str, const TimestampVal& tv) {
-  return DateTruncImpl(context, tv, unit_str);
-}
-
-static int64_t ExtractMillisecond(const time_duration& time) {
-  // Fractional seconds are nanoseconds because Boost is configured
-  // to use nanoseconds precision
-  return time.fractional_seconds() / (NANOS_PER_MICRO * MICROS_PER_MILLI)
-       + time.seconds() * MILLIS_PER_SEC;
-}
-
-// Maps the user facing name of a unit to a TExtractField
-// Returns the TExtractField for the given unit
-TExtractField::type StrToExtractField(FunctionContext* ctx, const StringVal& unit_str) {
-  StringVal unit = UdfBuiltins::Lower(ctx, unit_str);
-  if (UNLIKELY(unit.is_null)) return TExtractField::INVALID_FIELD;
-  if (unit == "year") return TExtractField::YEAR;
-  if (unit == "quarter") return TExtractField::QUARTER;
-  if (unit == "month") return TExtractField::MONTH;
-  if (unit == "day") return TExtractField::DAY;
-  if (unit == "hour") return TExtractField::HOUR;
-  if (unit == "minute") return TExtractField::MINUTE;
-  if (unit == "second") return TExtractField::SECOND;
-  if (unit == "millisecond") return TExtractField::MILLISECOND;
-  if (unit == "epoch") return TExtractField::EPOCH;
-  return TExtractField::INVALID_FIELD;
-}
-
-BigIntVal UdfBuiltins::Extract(FunctionContext* context, const StringVal& unit_str,
-    const TimestampVal& tv) {
-  // resolve extract_field using the prepared state if possible, o.w. parse now
-  // ExtractPrepare() can only parse extract_field if user passes it as a string literal
-  if (tv.is_null) return BigIntVal::null();
-
-  TExtractField::type field;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
-  if (state != NULL) {
-    field = *reinterpret_cast<TExtractField::type*>(state);
-  } else {
-    field = StrToExtractField(context, unit_str);
-    if (field == TExtractField::INVALID_FIELD) {
-      string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
-      context->SetError(Substitute("invalid extract field: $0", string_unit).c_str());
-      return BigIntVal::null();
-    }
-  }
-
-  const date& orig_date = *reinterpret_cast<const date*>(&tv.date);
-  const time_duration& time = *reinterpret_cast<const time_duration*>(&tv.time_of_day);
-
-  switch (field) {
-    case TExtractField::YEAR:
-    case TExtractField::QUARTER:
-    case TExtractField::MONTH:
-    case TExtractField::DAY:
-      if (orig_date.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::HOUR:
-    case TExtractField::MINUTE:
-    case TExtractField::SECOND:
-    case TExtractField::MILLISECOND:
-      if (time.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::EPOCH:
-      if (time.is_special() || orig_date.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::INVALID_FIELD:
-      DCHECK(false);
-  }
-
-  switch (field) {
-    case TExtractField::YEAR: {
-      return BigIntVal(orig_date.year());
-    }
-    case TExtractField::QUARTER: {
-      int m = orig_date.month();
-      return BigIntVal((m - 1) / 3 + 1);
-    }
-    case TExtractField::MONTH: {
-      return BigIntVal(orig_date.month());
-    }
-    case TExtractField::DAY: {
-      return BigIntVal(orig_date.day());
-    }
-    case TExtractField::HOUR: {
-      return BigIntVal(time.hours());
-    }
-    case TExtractField::MINUTE: {
-      return BigIntVal(time.minutes());
-    }
-    case TExtractField::SECOND: {
-      return BigIntVal(time.seconds());
-    }
-    case TExtractField::MILLISECOND: {
-      return BigIntVal(ExtractMillisecond(time));
-    }
-    case TExtractField::EPOCH: {
-      ptime epoch_date(date(1970, 1, 1), time_duration(0, 0, 0));
-      ptime cur_date(orig_date, time);
-      time_duration diff = cur_date - epoch_date;
-      return BigIntVal(diff.total_seconds());
-    }
-    default: {
-      DCHECK(false) << field;
-      return BigIntVal::null();
-    }
-  }
-}
-
-BigIntVal UdfBuiltins::Extract(FunctionContext* context, const TimestampVal& tv,
-    const StringVal& unit_str) {
-  return Extract(context, unit_str, tv);
-}
-
-void UdfBuiltins::ExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope, int unit_idx) {
-  // Parse the unit up front if we can, otherwise do it on the fly in Extract()
-  if (ctx->IsArgConstant(unit_idx)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(unit_idx));
-    TExtractField::type field = StrToExtractField(ctx, *unit_str);
-    if (field == TExtractField::INVALID_FIELD) {
-      string string_field(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute("invalid extract field: $0", string_field).c_str());
-    } else {
-      TExtractField::type* state = ctx->Allocate<TExtractField::type>();
-      RETURN_IF_NULL(ctx, state);
-      *state = field;
-      ctx->SetFunctionState(scope, state);
-    }
-  }
-}
-
-void UdfBuiltins::ExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  ExtractPrepare(ctx, scope, 0);
-}
-
-void UdfBuiltins::SwappedExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  ExtractPrepare(ctx, scope, 1);
-}
-
-void UdfBuiltins::ExtractClose(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
-}
-
 bool ValidateMADlibVector(FunctionContext* context, const StringVal& arr) {
   if (arr.ptr == NULL) {
     context->SetError("MADlib vector is null");
@@ -361,3 +198,135 @@ StringVal UdfBuiltins::DecodeVector(FunctionContext* context, const StringVal& a
   InplaceDoubleDecode(reinterpret_cast<char*>(result.ptr), arr.len);
   return result;
 }
+
+namespace {
+
+/// Used for closing TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+void CloseImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  void* state = ctx->GetFunctionState(scope);
+  ctx->Free(reinterpret_cast<uint8_t*>(state));
+  ctx->SetFunctionState(scope, nullptr);
+}
+
+}
+
+void UdfBuiltins::TruncForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return TruncForTimestampPrepareImpl(ctx, scope);
+}
+
+TimestampVal UdfBuiltins::TruncForTimestamp(FunctionContext* context,
+    const TimestampVal& tv, const StringVal &unit_str) {
+  return TruncForTimestampImpl(context, tv, unit_str);
+}
+
+void UdfBuiltins::TruncForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::TruncForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return TruncForDatePrepareImpl(ctx, scope);
+}
+
+DateVal UdfBuiltins::TruncForDate(FunctionContext* context, const DateVal& dv,
+    const StringVal &unit_str) {
+  return TruncForDateImpl(context, dv, unit_str);
+}
+
+void UdfBuiltins::TruncForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DateTruncForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return DateTruncForTimestampPrepareImpl(ctx, scope);
+}
+
+TimestampVal UdfBuiltins::DateTruncForTimestamp(FunctionContext* context,
+    const StringVal &unit_str, const TimestampVal& tv) {
+  return DateTruncForTimestampImpl(context, unit_str, tv);
+}
+
+void UdfBuiltins::DateTruncForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DateTruncForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return DateTruncForDatePrepareImpl(ctx, scope);
+}
+
+DateVal UdfBuiltins::DateTruncForDate(FunctionContext* context, const StringVal &unit_str,
+    const DateVal& dv) {
+  return DateTruncForDateImpl(context, unit_str, dv);
+}
+
+void UdfBuiltins::DateTruncForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::ExtractForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  ExtractForTimestampPrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::ExtractForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+    const StringVal& unit_str) {
+  return ExtractForTimestampImpl(ctx, tv, unit_str);
+}
+
+void UdfBuiltins::ExtractForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::ExtractForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  ExtractForDatePrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::ExtractForDate(FunctionContext* ctx, const DateVal& dv,
+    const StringVal& unit_str) {
+  return ExtractForDateImpl(ctx, dv, unit_str);
+}
+
+void UdfBuiltins::ExtractForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DatePartForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  DatePartForTimestampPrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::DatePartForTimestamp(FunctionContext* ctx,
+    const StringVal& unit_str, const TimestampVal& tv) {
+  return DatePartForTimestampImpl(ctx, unit_str, tv);
+}
+
+void UdfBuiltins::DatePartForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DatePartForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  DatePartForDatePrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::DatePartForDate(FunctionContext* ctx, const StringVal& unit_str,
+    const DateVal& dv) {
+  return DatePartForDateImpl(ctx, unit_str, dv);
+}
+
+void UdfBuiltins::DatePartForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
diff --git a/be/src/exprs/udf-builtins.cc b/be/src/exprs/udf-builtins.cc
index 61594ed..786e9cf 100644
--- a/be/src/exprs/udf-builtins.cc
+++ b/be/src/exprs/udf-builtins.cc
@@ -20,7 +20,10 @@
 
 #include "exprs/udf-builtins.h"
 
+#include <gutil/walltime.h>
+
 #include "gen-cpp/Exprs_types.h"
+#include "runtime/date-value.h"
 #include "runtime/runtime-state.h"
 #include "runtime/timestamp-value.h"
 #include "udf/udf-internal.h"
@@ -61,6 +64,7 @@ enum class TruncUnit {
 
 // Put non-exported functions in anonymous namespace to encourage inlining.
 namespace {
+
 // Returns the most recent date, no later than orig_date, which is on week_day
 // week_day: 0==Sunday, 1==Monday, ...
 date GoBackToWeekday(const date& orig_date, int week_day) {
@@ -245,7 +249,7 @@ TimestampValue TruncSecond(const date& orig_date, const time_duration& orig_time
 }
 
 // Truncate parts of milliseconds
-TimestampValue TruncMilliSeconds(const date& orig_date, const time_duration& orig_time) {
+TimestampValue TruncMilliseconds(const date& orig_date, const time_duration& orig_time) {
   time_duration new_time(orig_time.hours(), orig_time.minutes(), orig_time.seconds());
   // Fractional seconds are nanoseconds because Boost is configured to use nanoseconds
   // precision.
@@ -255,7 +259,7 @@ TimestampValue TruncMilliSeconds(const date& orig_date, const time_duration& ori
 }
 
 // Truncate parts of microseconds
-TimestampValue TruncMicroSeconds(const date& orig_date, const time_duration& orig_time) {
+TimestampValue TruncMicroseconds(const date& orig_date, const time_duration& orig_time) {
   time_duration new_time(orig_time.hours(), orig_time.minutes(), orig_time.seconds());
   // Fractional seconds are nanoseconds because Boost is configured to use nanoseconds
   // precision.
@@ -264,9 +268,9 @@ TimestampValue TruncMicroSeconds(const date& orig_date, const time_duration& ori
   return TimestampValue(orig_date, new_time);
 }
 
-// used by both Trunc and DateTrunc functions to perform the truncation
+// Used by both TRUNC and DATE_TRUNC functions to perform the truncation
 TimestampVal DoTrunc(
-    const TimestampValue ts, TruncUnit trunc_unit, FunctionContext* context) {
+    const TimestampValue& ts, TruncUnit trunc_unit, FunctionContext* ctx) {
   const date& orig_date = ts.date();
   const time_duration& orig_time = ts.time();
   TimestampValue ret;
@@ -354,115 +358,528 @@ TimestampVal DoTrunc(
       ret = TruncSecond(orig_date, orig_time);
       break;
     case TruncUnit::MILLISECONDS:
-      ret = TruncMilliSeconds(orig_date, orig_time);
+      ret = TruncMilliseconds(orig_date, orig_time);
       break;
     case TruncUnit::MICROSECONDS:
-      ret = TruncMicroSeconds(orig_date, orig_time);
+      ret = TruncMicroseconds(orig_date, orig_time);
       break;
     default:
       // internal error: implies StrToTruncUnit out of sync with this switch
-      context->SetError("truncate unit not supported");
+      ctx->SetError("truncate unit not supported");
       return TimestampVal::null();
   }
 
   ret.ToTimestampVal(&ret_val);
   return ret_val;
 }
-}
 
-TimestampVal UdfBuiltins::TruncImpl(
-    FunctionContext* context, const TimestampVal& tv, const StringVal& unit_str) {
-  if (tv.is_null) return TimestampVal::null();
-  TimestampValue ts = TimestampValue::FromTimestampVal(tv);
+// Returns the most recent date, no later than 'orig_date', which is on 'week_day'
+// 'week_day' is in [0, 6]; 0 = Monday, 6 = Sunday.
+DateValue GoBackToWeekday(const DateValue& orig_date, int week_day) {
+  DCHECK(orig_date.IsValid());
+  DCHECK(week_day >= 0 && week_day <= 6);
 
-  // resolve trunc_unit using the prepared state if possible, o.w. parse now
-  // TruncPrepare() can only parse trunc_unit if user passes it as a string literal
-  // TODO: it would be nice to resolve the branch before codegen so we can optimise
-  // this better.
-  TruncUnit trunc_unit;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
-  if (state != NULL) {
-    trunc_unit = *reinterpret_cast<TruncUnit*>(state);
+  // Week days are in [0, 6]; 0 = Monday, 6 = Sunday.
+  int current_week_day = orig_date.WeekDay();
+  DCHECK(current_week_day >= 0 && current_week_day <= 6);
+
+  if (current_week_day == week_day) {
+    return orig_date;
+  } else if (current_week_day > week_day) {
+    return orig_date.AddDays(week_day - current_week_day);
   } else {
-    trunc_unit = StrToTruncUnit(context, unit_str);
-    if (trunc_unit == TruncUnit::UNIT_INVALID) {
-      string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
-      context->SetError(Substitute("Invalid Truncate Unit: $0", string_unit).c_str());
-      return TimestampVal::null();
+    return orig_date.AddDays(week_day - current_week_day - 7);
+  }
+}
+
+// Used by both TRUNC and DATE_TRUNC functions to perform the truncation
+DateVal DoTrunc(const DateValue& date, TruncUnit trunc_unit, FunctionContext* ctx) {
+  if (!date.IsValid()) return DateVal::null();
+
+  DCHECK(trunc_unit != TruncUnit::UNIT_INVALID
+      && trunc_unit != TruncUnit::MICROSECONDS
+      && trunc_unit != TruncUnit::HOUR
+      && trunc_unit != TruncUnit::MINUTE
+      && trunc_unit != TruncUnit::SECOND
+      && trunc_unit != TruncUnit::MILLISECONDS);
+
+  DateValue ret;
+
+  switch(trunc_unit) {
+    case TruncUnit::YEAR: {
+      int year;
+      discard_result(date.ToYear(&year));
+      ret = DateValue(year, 1, 1);
+      break;
+    }
+    case TruncUnit::QUARTER: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = DateValue(year, BitUtil::RoundDown(month - 1, 3) + 1, 1);
+      break;
+    }
+    case TruncUnit::MONTH: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = DateValue(year, month, 1);
+      break;
+    }
+    case TruncUnit::DAY: {
+      ret = date;
+      break;
+    }
+    case TruncUnit::WW: {
+      int year;
+      discard_result(date.ToYear(&year));
+      ret = GoBackToWeekday(date, DateValue(year, 1, 1).WeekDay());
+      break;
+    }
+    case TruncUnit::W: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = GoBackToWeekday(date, DateValue(year, month, 1).WeekDay());
+      break;
+    }
+    case TruncUnit::DAY_OF_WEEK: {
+      // Date of the previous Monday
+      ret = GoBackToWeekday(date, 0);
+      break;
+    }
+    case TruncUnit::WEEK: {
+      // ISO-8601 week starts on monday. go back to monday
+      ret = GoBackToWeekday(date, 0);
+      break;
+    }
+    case TruncUnit::MILLENNIUM: {
+      int year;
+      discard_result(date.ToYear(&year));
+      if (year <= 0) return DateVal::null();
+      // First year of current millennium is 2001
+      ret = DateValue((year - 1) / 1000 * 1000 + 1, 1, 1);
+      break;
+    }
+    case TruncUnit::CENTURY: {
+      int year;
+      discard_result(date.ToYear(&year));
+      if (year <= 0) return DateVal::null();
+      // First year of current century is 2001
+      ret = DateValue((year - 1) / 100 * 100 + 1, 1, 1);
+      break;
+    }
+    case TruncUnit::DECADE: {
+      int year;
+      // Decades start with years ending in '0'.
+      discard_result(date.ToYear(&year));
+      ret = DateValue(year / 10 * 10, 1, 1);
+      break;
     }
+    default:
+      // internal error: implies StrToTruncUnit out of sync with this switch
+      ctx->SetError("truncate unit not supported");
+      return DateVal::null();
   }
-  return DoTrunc(ts, trunc_unit, context);
+
+  return ret.ToDateVal();
 }
 
-void UdfBuiltins::TruncPrepare(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  // Parse the unit up front if we can, otherwise do it on the fly in Trunc()
-  if (ctx->IsArgConstant(1)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(1));
-    TruncUnit trunc_unit = StrToTruncUnit(ctx, *unit_str);
-    if (trunc_unit == TruncUnit::UNIT_INVALID) {
-      string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute("Invalid Truncate Unit: $0", string_unit).c_str());
-    } else {
-      TruncUnit* state = ctx->Allocate<TruncUnit>();
-      RETURN_IF_NULL(ctx, state);
-      *state = trunc_unit;
-      ctx->SetFunctionState(scope, state);
+// Maps the user facing name of a unit to a TExtractField
+// Returns the TExtractField for the given unit
+TExtractField::type StrToExtractField(FunctionContext* ctx,
+    const StringVal& unit_str) {
+  StringVal unit = UdfBuiltins::Lower(ctx, unit_str);
+  if (UNLIKELY(unit.is_null)) return TExtractField::INVALID_FIELD;
+  if (unit == "year") return TExtractField::YEAR;
+  if (unit == "quarter") return TExtractField::QUARTER;
+  if (unit == "month") return TExtractField::MONTH;
+  if (unit == "day") return TExtractField::DAY;
+  if (unit == "hour") return TExtractField::HOUR;
+  if (unit == "minute") return TExtractField::MINUTE;
+  if (unit == "second") return TExtractField::SECOND;
+  if (unit == "millisecond") return TExtractField::MILLISECOND;
+  if (unit == "epoch") return TExtractField::EPOCH;
+  return TExtractField::INVALID_FIELD;
+}
+
+static int64_t ExtractMillisecond(const time_duration& time) {
+  // Fractional seconds are nanoseconds because Boost is configured
+  // to use nanoseconds precision
+  return time.fractional_seconds() / (NANOS_PER_MICRO * MICROS_PER_MILLI)
+       + time.seconds() * MILLIS_PER_SEC;
+}
+
+// Used by both EXTRACT and DATE_PART functions to perform field extraction.
+BigIntVal DoExtract(const TimestampValue& tv, TExtractField::type field,
+    FunctionContext* ctx) {
+  switch (field) {
+    case TExtractField::YEAR:
+    case TExtractField::QUARTER:
+    case TExtractField::MONTH:
+    case TExtractField::DAY:
+      if (!tv.HasDate()) return BigIntVal::null();
+      break;
+    case TExtractField::HOUR:
+    case TExtractField::MINUTE:
+    case TExtractField::SECOND:
+    case TExtractField::MILLISECOND:
+      if (!tv.HasTime()) return BigIntVal::null();
+      break;
+    case TExtractField::EPOCH:
+      if (!tv.HasDateAndTime()) return BigIntVal::null();
+      break;
+    case TExtractField::INVALID_FIELD:
+      DCHECK(false);
+  }
+
+  const date& orig_date = tv.date();
+  const time_duration& time = tv.time();
+
+  switch (field) {
+    case TExtractField::YEAR: {
+      return BigIntVal(orig_date.year());
+    }
+    case TExtractField::QUARTER: {
+      int m = orig_date.month();
+      return BigIntVal((m - 1) / 3 + 1);
+    }
+    case TExtractField::MONTH: {
+      return BigIntVal(orig_date.month());
+    }
+    case TExtractField::DAY: {
+      return BigIntVal(orig_date.day());
+    }
+    case TExtractField::HOUR: {
+      return BigIntVal(time.hours());
+    }
+    case TExtractField::MINUTE: {
+      return BigIntVal(time.minutes());
+    }
+    case TExtractField::SECOND: {
+      return BigIntVal(time.seconds());
+    }
+    case TExtractField::MILLISECOND: {
+      return BigIntVal(ExtractMillisecond(time));
+    }
+    case TExtractField::EPOCH: {
+      ptime epoch_date(date(1970, 1, 1), time_duration(0, 0, 0));
+      ptime cur_date(orig_date, time);
+      time_duration diff = cur_date - epoch_date;
+      return BigIntVal(diff.total_seconds());
+    }
+    default: {
+      // internal error: implies StrToExtractField out of sync with this switch
+      ctx->SetError("extract unit not supported");
+      return BigIntVal::null();
     }
   }
 }
 
-void UdfBuiltins::TruncClose(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
+// Used by both EXTRACT and DATE_PART functions to perform field extraction.
+BigIntVal DoExtract(const DateValue& dv, TExtractField::type field,
+    FunctionContext* ctx) {
+  if (!dv.IsValid()) return BigIntVal::null();
+
+  DCHECK(field != TExtractField::INVALID_FIELD
+      && field != TExtractField::HOUR
+      && field != TExtractField::MINUTE
+      && field != TExtractField::SECOND
+      && field != TExtractField::MILLISECOND
+      && field != TExtractField::EPOCH);
+
+  switch (field) {
+    case TExtractField::YEAR: {
+      int year;
+      discard_result(dv.ToYear(&year));
+      return BigIntVal(year);
+    }
+    case TExtractField::QUARTER: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal((month - 1) / 3 + 1);
+    }
+    case TExtractField::MONTH: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal(month);
+    }
+    case TExtractField::DAY: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal(day);
+    }
+    default: {
+      // internal error: implies StrToExtractField out of sync with this switch
+      ctx->SetError("extract unit not supported");
+      return BigIntVal::null();
+    }
+  }
+}
+
+inline TimestampValue FromVal(const TimestampVal& val) {
+  return TimestampValue::FromTimestampVal(val);
+}
+
+inline DateValue FromVal(const DateVal& val) {
+  return DateValue::FromDateVal(val);
+}
+
+inline bool IsTimeOfDayUnit(TruncUnit unit) {
+  return (unit == TruncUnit::HOUR
+      || unit == TruncUnit::MINUTE
+      || unit == TruncUnit::SECOND
+      || unit == TruncUnit::MILLISECONDS
+      || unit == TruncUnit::MICROSECONDS);
+}
+
+inline bool IsTimeOfDayUnit(TExtractField::type unit) {
+  return (unit == TExtractField::HOUR
+      || unit == TExtractField::MINUTE
+      || unit == TExtractField::SECOND
+      || unit == TExtractField::MILLISECOND
+      || unit == TExtractField::EPOCH);
 }
 
-TimestampVal UdfBuiltins::DateTruncImpl(
-    FunctionContext* context, const TimestampVal& tv, const StringVal& unit_str) {
-  if (tv.is_null) return TimestampVal::null();
-  TimestampValue ts = TimestampValue::FromTimestampVal(tv);
+inline bool IsInvalidUnit(TruncUnit unit) {
+  return (unit == TruncUnit::UNIT_INVALID);
+}
+
+inline bool IsInvalidUnit(TExtractField::type unit) {
+  return (unit == TExtractField::INVALID_FIELD);
+}
 
-  // resolve date_trunc_unit using the prepared state if possible, o.w. parse now
-  // DateTruncPrepare() can only parse trunc_unit if user passes it as a string literal
-  TruncUnit date_trunc_unit;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
+/// Used for TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+/// ALLOW_TIME_OF_DAY_UNIT: true iff the built-in function call accepts time-of-day units.
+/// UdfType: udf type the built-in function works with.
+/// InternalType: Impla's internal type that corresponds to UdfType.
+/// ReturnUdfType: The built-in function's return type.
+/// UnitType: type to represent unit values.
+/// to_unit: function to parse unit strings.
+/// do_func: function to implement the built-in function.
+/// func_descr: description of the built-in function.
+/// unit_descr: description of the unit parameter.
+template <
+    bool ALLOW_TIME_OF_DAY_UNIT,
+    typename UdfType,
+    typename InternalType,
+    typename ReturnUdfType,
+    typename UnitType,
+    UnitType to_unit(FunctionContext*, const StringVal&),
+    ReturnUdfType do_func(const InternalType&, UnitType, FunctionContext*)>
+ReturnUdfType ExtractTruncFuncTempl(FunctionContext* ctx, const UdfType& val,
+    const StringVal& unit_str, const string& func_descr, const string& unit_descr) {
+  if (val.is_null) return ReturnUdfType::null();
+
+  // resolve 'unit' using the prepared state if possible, o.w. parse now
+  // ExtractTruncFuncPrepareTempl() can only parse unit if user passes it as a string
+  // literal
+  // TODO: it would be nice to resolve the branch before codegen so we can optimise
+  // this better.
+  UnitType unit;
+  void* state = ctx->GetFunctionState(FunctionContext::THREAD_LOCAL);
   if (state != NULL) {
-    date_trunc_unit = *reinterpret_cast<TruncUnit*>(state);
+    unit = *reinterpret_cast<UnitType*>(state);
   } else {
-    date_trunc_unit = StrToDateTruncUnit(context, unit_str);
-    if (date_trunc_unit == TruncUnit::UNIT_INVALID) {
+    unit = to_unit(ctx, unit_str);
+    if (!ALLOW_TIME_OF_DAY_UNIT && IsTimeOfDayUnit(unit)) {
       string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
-      context->SetError(
-          Substitute("Invalid Date Truncate Unit: $0", string_unit).c_str());
-      return TimestampVal::null();
+      ctx->SetError(Substitute(
+          "Unsupported $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+      return ReturnUdfType::null();
+    } else if (IsInvalidUnit(unit)) {
+      string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
+      ctx->SetError(Substitute(
+          "Invalid $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+      return ReturnUdfType::null();
     }
   }
-  return DoTrunc(ts, date_trunc_unit, context);
+  return do_func(FromVal(val), unit, ctx);
 }
 
-void UdfBuiltins::DateTruncPrepare(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  // Parse the unit up front if we can, otherwise do it on the fly in DateTrunc()
-  if (ctx->IsArgConstant(0)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(0));
-    TruncUnit date_trunc_unit = StrToDateTruncUnit(ctx, *unit_str);
-    if (date_trunc_unit == TruncUnit::UNIT_INVALID) {
+/// Does the preparation for TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+/// ALLOW_TIME_OF_DAY_UNIT: true iff the built-in function call accepts time-of-day units.
+/// UNIT_IDX: indicates which parameter of the function call is the unit parameter.
+/// UnitType: type to represent unit values.
+/// to_unit: function to parse unit strings.
+/// func_descr: description of the built-in function.
+/// unit_descr: description of the unit parameter.
+template <
+    bool ALLOW_TIME_OF_DAY_UNIT,
+    int UNIT_IDX,
+    typename UnitType,
+    UnitType to_unit(FunctionContext*, const StringVal&)>
+void ExtractTruncFuncPrepareTempl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope,
+    const string& func_descr, const string& unit_descr) {
+  // Parse the unit up front if we can, otherwise do it on the fly in trunc_templ()
+  if (ctx->IsArgConstant(UNIT_IDX)) {
+    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(UNIT_IDX));
+    UnitType unit = to_unit(ctx, *unit_str);
+    if (!ALLOW_TIME_OF_DAY_UNIT && IsTimeOfDayUnit(unit)) {
+      string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
+      ctx->SetError(Substitute(
+          "Unsupported $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+    } else if (IsInvalidUnit(unit)) {
       string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute("Invalid Date Truncate Unit: $0", string_unit).c_str());
+      ctx->SetError(Substitute(
+          "Invalid $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
     } else {
-      TruncUnit* state = ctx->Allocate<TruncUnit>();
+      UnitType* state = ctx->Allocate<UnitType>();
       RETURN_IF_NULL(ctx, state);
-      *state = date_trunc_unit;
+      *state = unit;
       ctx->SetFunctionState(scope, state);
     }
   }
 }
 
-void UdfBuiltins::DateTruncClose(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
+}
+
+void UdfBuiltins::TruncForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      1,
+      TruncUnit,
+      StrToTruncUnit>(ctx, scope, "Truncate", "Unit");
+}
+
+TimestampVal UdfBuiltins::TruncForTimestampImpl(FunctionContext* ctx,
+    const TimestampVal& tv, const StringVal &unit_str) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      TimestampVal,
+      TruncUnit,
+      StrToTruncUnit,
+      DoTrunc>(ctx, tv, unit_str, "Truncate", "Unit");
+}
+
+void UdfBuiltins::TruncForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      1,
+      TruncUnit,
+      StrToTruncUnit>(ctx, scope, "Truncate", "Unit");
+}
+
+DateVal UdfBuiltins::TruncForDateImpl(FunctionContext* ctx, const DateVal& dv,
+    const StringVal &unit_str) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      DateVal,
+      TruncUnit,
+      StrToTruncUnit,
+      DoTrunc>(ctx, dv, unit_str, "Truncate", "Unit");
+}
+
+void UdfBuiltins::DateTruncForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      0,
+      TruncUnit,
+      StrToDateTruncUnit>(ctx, scope, "Date Truncate", "Unit");
+}
+
+TimestampVal UdfBuiltins::DateTruncForTimestampImpl(FunctionContext* ctx,
+    const StringVal &unit_str, const TimestampVal& tv) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      TimestampVal,
+      TruncUnit,
+      StrToDateTruncUnit,
+      DoTrunc>(ctx, tv, unit_str, "Date Truncate", "Unit");
+}
+
+void UdfBuiltins::DateTruncForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      0,
+      TruncUnit,
+      StrToDateTruncUnit>(ctx, scope, "Date Truncate", "Unit");
+}
+
+DateVal UdfBuiltins::DateTruncForDateImpl(FunctionContext* ctx, const StringVal &unit_str,
+    const DateVal& dv) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      DateVal,
+      TruncUnit,
+      StrToDateTruncUnit,
+      DoTrunc>(ctx, dv, unit_str, "Date Truncate", "Unit");
+}
+
+void UdfBuiltins::ExtractForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      1,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Extract", "Field");
+}
+
+BigIntVal UdfBuiltins::ExtractForTimestampImpl(FunctionContext* ctx,
+    const TimestampVal& tv, const StringVal& unit_str) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, tv, unit_str, "Extract", "Field");
+}
+
+void UdfBuiltins::ExtractForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      1,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Extract", "Field");
+}
+
+BigIntVal UdfBuiltins::ExtractForDateImpl(FunctionContext* ctx, const DateVal& dv,
+    const StringVal& unit_str) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, dv, unit_str, "Extract", "Field");
+}
+
+void UdfBuiltins::DatePartForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      0,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Date Part", "Field");
+}
+
+BigIntVal UdfBuiltins::DatePartForTimestampImpl(FunctionContext* ctx,
+    const StringVal& unit_str, const TimestampVal& tv) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, tv, unit_str, "Date Part", "Field");
+}
+
+void UdfBuiltins::DatePartForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      0,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Date Part", "Field");
+}
+
+BigIntVal UdfBuiltins::DatePartForDateImpl(FunctionContext* ctx,
+    const StringVal& unit_str, const DateVal& dv) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, dv, unit_str, "Date Part", "Field");
 }
diff --git a/be/src/exprs/udf-builtins.h b/be/src/exprs/udf-builtins.h
index 8aec873..430a0fa 100644
--- a/be/src/exprs/udf-builtins.h
+++ b/be/src/exprs/udf-builtins.h
@@ -33,6 +33,7 @@ using impala_udf::DoubleVal;
 using impala_udf::TimestampVal;
 using impala_udf::StringVal;
 using impala_udf::DecimalVal;
+using impala_udf::DateVal;
 
 /// Builtins written against the UDF interface. The builtins in the other files
 /// should be replaced to the UDF interface as well.
@@ -56,6 +57,7 @@ class UdfBuiltins {
   static BooleanVal IsNan(FunctionContext* context, const DoubleVal& val);
   static BooleanVal IsInf(FunctionContext* context, const DoubleVal& val);
 
+  /// This is for TRUNC(TIMESTAMP, STRING) function.
   /// Rounds (truncating down) a Timestamp to the specified unit.
   ///    Units:
   ///    CC, SCC : One greater than the first two digits of
@@ -63,25 +65,36 @@ class UdfBuiltins {
   ///    SYYYY, YYYY, YEAR, SYEAR, YYY, YY, Y : Current Year
   ///    Q : Quarter
   ///    MONTH, MON, MM, RM : Month
-  ///    WW : Same day of the week as the first day of the year
-  ///    W : Same day of the week as the first day of the month
   ///    DDD, DD, J : Day
   ///    DAY, DY, D : Starting day of the week
+  ///    WW : Truncates to the most recent date, no later than 'tv', which is on the same
+  ///         day of the week as the first day of year.
+  ///    W : Truncates to the most recent date, no later than 'tv', which is on the same
+  ///        day of the week as the first day of month.
   ///    HH, HH12, HH24 : Hour
   ///    MI : Minute
   ///
   ///    Reference:
   ///    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions201.htm
-  static TimestampVal Trunc(FunctionContext* context, const TimestampVal& date,
-      const StringVal& unit_str);
-  /// Implementation of Trunc, not cross-compiled.
-  static TimestampVal TruncImpl(FunctionContext* context, const TimestampVal& date,
-      const StringVal& unit_str);
-  static void TruncPrepare(FunctionContext* context,
+  static TimestampVal TruncForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+      const StringVal &unit_str);
+  static void TruncForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void TruncForTimestampClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
-  static void TruncClose(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
 
+  /// This for TRUNC(DATE, STRING) function.
+  /// Rounds (truncating down) a Date to the specified unit.
+  /// Works as 'TruncForTimestamp' but doesn't accept time of day units: HH, HH12, HH24,
+  /// MI.
+  static DateVal TruncForDate(FunctionContext* ctx, const DateVal& dv,
+      const StringVal &unit_str);
+  static void TruncForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void TruncForDateClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This for DATE_TRUNC(STRING, TIMESTAMP) function.
   /// Rounds (truncating down) a Timestamp to the specified unit.
   ///    Units:
   ///    MILLENNIUM: The millennium number.
@@ -96,44 +109,69 @@ class UdfBuiltins {
   ///    SECOND: The seconds field (0–59).
   ///    MILLISECONDS: The milliseconds fraction in the seconds.
   ///    MICROSECONDS: The microseconds fraction in the seconds.
-
+  ///
   ///    Reference:
   ///    https://my.vertica.com/docs/8.1.x/HTML/index.htm#Authoring/
   ///       SQLReferenceManual/Functions/Date-Time/DATE_TRUNC.htm
-  static TimestampVal DateTrunc(
-      FunctionContext* context, const StringVal& unit_str, const TimestampVal& date);
-  /// Implementation of DateTrunc, not cross-compiled.
-  static TimestampVal DateTruncImpl(
-      FunctionContext* context, const TimestampVal& date, const StringVal& unit_str);
-  static void DateTruncPrepare(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
-  static void DateTruncClose(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
+  static TimestampVal DateTruncForTimestamp(FunctionContext* ctx,
+      const StringVal &unit_str, const TimestampVal& tv);
+  static void DateTruncForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void DateTruncForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This is for DATE_TRUNC(STRING, DATE) function.
+  /// Rounds (truncating down) a Date to the specified unit.
+  /// Works as 'DateTruncForTimestamp' but doesn't accept time of day units: HOUR, MINUTE,
+  /// SECOND, MILLISECONDS, MICROSECONDS.
+  static DateVal DateTruncForDate(FunctionContext* ctx, const StringVal &unit_str,
+      const DateVal& dv);
+  static void DateTruncForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void DateTruncForDateClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
 
+  /// This is for the EXTRACT(TIMESTAMP, STRING) and EXTRACT(TIMEUNIT FROM TIMESTAMP)
+  /// functions.
   /// Returns a single field from a timestamp
   ///    Fields:
   ///      YEAR, MONTH, DAY, HOUR, MINUTE, SECOND, MILLISECOND, EPOCH
-  ///    Reference:
-  ///    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
-  ///
-  /// This is used by the DATE_PART function.
-  static BigIntVal Extract(FunctionContext* context, const StringVal& field_str,
-      const TimestampVal& date);
+  /// Reference:
+  /// http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
+  static void ExtractForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+      const StringVal& unit_str);
+  static void ExtractForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
 
-  /// This is for the EXTRACT(Timestamp, String) and EXTRACT(Timeunit FROM
-  /// Timestamp) functions.
-  static BigIntVal Extract(FunctionContext* context, const TimestampVal& date,
-      const StringVal& field_str);
-  /// This is used by the DATE_PART function.
-  static void ExtractPrepare(FunctionContext* context,
+  /// This is for the EXTRACT(DATE, STRING) and EXTRACT(TIMEUNIT FROM DATE)
+  /// functions.
+  /// Works as 'ExtractForTimestamp' but doesn't accept time of day fields: HOUR, MINUTE,
+  /// SECOND, MILLISECOND, EPOCH.
+  static void ExtractForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForDate(FunctionContext* ctx, const DateVal& dv,
+      const StringVal& unit_str);
+  static void ExtractForDateClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
 
-  /// This is for the EXTRACT(Timestamp, String) and EXTRACT(Timeunit FROM
-  /// Timestamp) functions.
-  static void SwappedExtractPrepare(FunctionContext* context,
+  /// This is for DATE_PART(STRING, TIMESTAMP) function.
+  /// Similar to 'ExtractForTimestamp' with the argument order reversed.
+  static void DatePartForTimestampPrepare(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
-  /// This is used by both EXTRACT and DATE_PART
-  static void ExtractClose(FunctionContext* context,
+  static BigIntVal DatePartForTimestamp(FunctionContext* ctx, const StringVal& unit_str,
+      const TimestampVal& tv);
+  static void DatePartForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This is for DATE_PART(STRING, DATE) function.
+  /// Similar to 'ExtractForDate' with the argument order reversed.
+  static void DatePartForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForDate(FunctionContext* ctx, const StringVal& unit_str,
+      const DateVal& dv);
+  static void DatePartForDateClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
 
   /// Converts a set of doubles to double[] stored as a StringVal
@@ -159,11 +197,49 @@ class UdfBuiltins {
   static StringVal DecodeVector(FunctionContext* context, const StringVal& arr);
 
  private:
-  /// Does the preparation for EXTRACT. The unit_idx parameter should indicate which
-  /// parameter of the EXTRACT call is the time unit param. DATE_PART will also use this
-  /// with a different unit_idx than EXTRACT.
-  static void ExtractPrepare(FunctionContext* context,
-      FunctionContext::FunctionStateScope scope, int unit_idx);
+  /// Implementation of TruncForTimestamp, not cross-compiled.
+  static void TruncForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static TimestampVal TruncForTimestampImpl(FunctionContext* context,
+      const TimestampVal& tv, const StringVal &unit_str);
+  /// Implementation of TruncForDate, not cross-compiled.
+  static void TruncForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static DateVal TruncForDateImpl(FunctionContext* context, const DateVal& dv,
+      const StringVal &unit_str);
+
+  /// Implementation of DateTruncForTimestamp, not cross-compiled.
+  static void DateTruncForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static TimestampVal DateTruncForTimestampImpl(FunctionContext* context,
+      const StringVal &unit_str, const TimestampVal& tv);
+  /// Implementation of DateTruncForDate, not cross-compiled.
+  static void DateTruncForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static DateVal DateTruncForDateImpl(FunctionContext* context, const StringVal &unit_str,
+      const DateVal& dv);
+
+  /// Implementation of ExtractForTimestamp, not cross-compiled.
+  static void ExtractForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForTimestampImpl(FunctionContext* context,
+      const TimestampVal& tv, const StringVal& unit_str);
+  /// Implementation of ExtractForDate, not cross-compiled.
+  static void ExtractForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForDateImpl(FunctionContext* context, const DateVal& dv,
+      const StringVal& unit_st);
+
+  /// Implementation of DatePartForTimestamp, not cross-compiled.
+  static void DatePartForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForTimestampImpl(FunctionContext* context,
+      const StringVal& unit_str, const TimestampVal& tv);
+  /// Implementation of DatePartForDate, not cross-compiled.
+  static void DatePartForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForDateImpl(FunctionContext* context,
+      const StringVal& unit_str, const DateVal& dv);
 };
 
 } // namespace impala
diff --git a/be/src/runtime/date-test.cc b/be/src/runtime/date-test.cc
index 95f90e3..0b8f804 100644
--- a/be/src/runtime/date-test.cc
+++ b/be/src/runtime/date-test.cc
@@ -593,6 +593,93 @@ TEST(DateTest, DateValueEdgeCases) {
   }
 }
 
+TEST(DateTest, AddDays) {
+  // Adding days to an invalid DateValue instance returns an invalid DateValue.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_FALSE(invalid_dv.AddDays(1).IsValid());
+
+  // AddDays works with 0, > 0 and < 0 number of days.
+  DateValue dv(2019, 5, 16);
+  EXPECT_EQ(DateValue(2019, 5, 17), dv.AddDays(1));
+  EXPECT_EQ(DateValue(2019, 5, 15), dv.AddDays(-1));
+  // May has 31 days, April has 30 days.
+  EXPECT_EQ(DateValue(2019, 6, 16), dv.AddDays(31));
+  EXPECT_EQ(DateValue(2019, 4, 16), dv.AddDays(-30));
+  // 2019 is not a leap year, 2020 is a leap year.
+  EXPECT_EQ(DateValue(2020, 5, 16), dv.AddDays(366));
+  EXPECT_EQ(DateValue(2018, 5, 16), dv.AddDays(-365));
+
+  // Test upper limit
+  dv = DateValue(9999, 12, 20);
+  EXPECT_EQ(DateValue(9999, 12, 31), dv.AddDays(11));
+  EXPECT_FALSE(dv.AddDays(12).IsValid());
+  EXPECT_FALSE(dv.AddDays(13).IsValid());
+
+  // Test lower limit
+  dv = DateValue(0, 1, 10);
+  EXPECT_EQ(DateValue(0, 1, 1), dv.AddDays(-9));
+  EXPECT_FALSE(dv.AddDays(-10).IsValid());
+  EXPECT_FALSE(dv.AddDays(-11).IsValid());
+
+  // Test leap year
+  dv = DateValue(2000, 2, 20);
+  EXPECT_EQ(DateValue(2000, 2, 28), dv.AddDays(8));
+  EXPECT_EQ(DateValue(2000, 2, 29), dv.AddDays(9));
+  EXPECT_EQ(DateValue(2000, 3, 1), dv.AddDays(10));
+
+  // Test non-leap year
+  dv = DateValue(2001, 2, 20);
+  EXPECT_EQ(DateValue(2001, 2, 28), dv.AddDays(8));
+  EXPECT_EQ(DateValue(2001, 3, 1), dv.AddDays(9));
+}
+
+TEST(DateTest, WeekDay) {
+  // WeekDay() returns -1 for invalid dates.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_EQ(-1, invalid_dv.WeekDay());
+
+  // 2019.05.01 is Wednesday.
+  DateValue dv(2019, 5, 1);
+  for (int i = 0; i <= 31; ++i) {
+    // 0 = Monday, 2 = Wednesday and 6 = Sunday.
+    EXPECT_EQ((i + 2) % 7, dv.AddDays(i).WeekDay());
+  }
+
+  // Test upper limit. 9999.12.31 is Friday.
+  EXPECT_EQ(4, DateValue(9999, 12, 31).WeekDay());
+
+  // Test lower limit.
+  // 0000.01.01 is Monday.
+  EXPECT_EQ(0, DateValue(1, 1, 1).WeekDay());
+  // 0000.01.01 is Saturday.
+  EXPECT_EQ(5, DateValue(0, 1, 1).WeekDay());
+}
+
+TEST(DateTest, ToYear) {
+  int year;
+
+  // Test that ToYear() returns false for invalid dates.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_FALSE(invalid_dv.ToYear(&year));
+
+  // Test that ToYear() returns the same year as ToYearMonthDay().
+  // The following loop iterates through all valid dates:
+  DateValue dv(0, 1, 1);
+  EXPECT_TRUE(dv.IsValid());
+  do {
+    int y, m, d;
+    EXPECT_TRUE(dv.ToYearMonthDay(&y, &m, &d));
+
+    EXPECT_TRUE(dv.ToYear(&year));
+    EXPECT_EQ(y, year);
+
+    dv = dv.AddDays(1);
+  } while (dv.IsValid());
+}
+
 }
 
 IMPALA_TEST_MAIN();
diff --git a/be/src/runtime/date-value.cc b/be/src/runtime/date-value.cc
index a9dd44e..54b31da 100644
--- a/be/src/runtime/date-value.cc
+++ b/be/src/runtime/date-value.cc
@@ -28,7 +28,8 @@
 namespace impala {
 
 namespace {
-const cctz::civil_day EPOCH_DATE(1970, 1, 1);
+const int EPOCH_YEAR = 1970;
+const cctz::civil_day EPOCH_DATE(EPOCH_YEAR, 1, 1);
 
 inline int32_t CalcDaysSinceEpoch(const cctz::civil_day& date) {
   return date - EPOCH_DATE;
@@ -97,6 +98,97 @@ bool DateValue::ToYearMonthDay(int* year, int* month, int* day) const {
   return true;
 }
 
+namespace {
+
+inline int32_t CalcFirstDayOfYearSinceEpoch(int year) {
+  int m400 = year % 400;
+  int m100 = m400 % 100;
+  int m4 = m100 % 4;
+
+  return (year - EPOCH_YEAR) * 365
+      + ((year - EPOCH_YEAR / 4 * 4 + ((m4 != 0) ? 4 - m4 : 0)) / 4 - 1)
+      - ((year - EPOCH_YEAR / 100 * 100 + ((m100 != 0) ? 100 - m100 : 0)) / 100 - 1)
+      + ((year - EPOCH_YEAR / 400 * 400 + ((m400 != 0) ? 400 - m400 : 0)) / 400 - 1);
+}
+
+}
+
+bool DateValue::ToYear(int* year) const {
+  DCHECK(year != nullptr);
+  if (UNLIKELY(!IsValid())) return false;
+
+  // This function was introduced to extract year of a DateValue efficiently.
+  // It will be fast for most days of the year and only slightly slower for days around
+  // the beginning and end of the year.
+  //
+  // Here's a quick explanation. Let's use the following notation:
+  // m400 = year % 400
+  // m100 = m400 % 100
+  // m4 = m100 % 4
+  //
+  // If 'days' is the number of days between 1970-01-01 and the first day of 'year'
+  // (excluding the endpoint), then the following is true:
+  // days == (year - 1970) * 365
+  //       + ((year - 1968 + ((m4 != 0) ? 4 - m4 : 0)) / 4 - 1)
+  //       - ((year - 1900 + ((m100 != 0) ? 100 - m100 : 0)) / 100 - 1)
+  //       + ((year - 1600 + ((m400 != 0) ? 400 - m400 : 0)) / 400 - 1)
+  //
+  // Reordering the equation we get:
+  // days * 400 == (year - 1970) * 365 * 400
+  //       + ((year - 1968) * 100 + ((m4 != 0) ? (4 - m4) * 100 : 0) - 400)
+  //       - ((year - 1900) * 4 + ((m100 != 0) ? (100 - m100) * 4 : 0) - 400)
+  //       + (year - 1600 + ((m400 != 0) ? 400 - m400 : 0) - 400)
+  //
+  // then:
+  // days * 400 == year * 146000 - 287620000
+  //       + (year * 100 - 196800 + ((m4 != 0) ? (4 - m4) * 100 : 0) - 400)
+  //       - (year * 4 - 7600 + ((m100 != 0) ? (100 - m100) * 4 : 0) - 400)
+  //       + (year - 1600 + ((m400 != 0) ? 400 - m400 : 0) - 400)
+  //
+  // which means that (A):
+  // year * 146097 == days * 400 + 287811200
+  //       - ((m4 != 0) ? (4 - m4) * 100 : 0)
+  //       + ((m100 != 0) ? (100 - m100) * 4 : 0)
+  //       - ((m400 != 0) ? 400 - m400 : 0)
+  //
+  // On the other hand, if
+  // f(year) = - ((m4 != 0) ? (4 - m4) * 100 : 0)
+  //           + ((m100 != 0) ? (100 - m100) * 4 : 0)
+  //           - ((m400 != 0) ? 400 - m400 : 0)
+  // and 'year' is in the [0, 9999] range, then it follows that (B):
+  // f(year) must fall into the [-591, 288] range.
+  //
+  // Finally, if we put (A) and (B) together we can conclude that 'year' must fall into
+  // the
+  // [ (days * 400 + 287811200 - 591) / 146097, (days * 400 + 287811200 + 288) / 146097 ]
+  // range.
+
+  int tmp = days_since_epoch_ * 400 + 287811200;
+  int first_year = (tmp - 591) / 146097;
+  int last_year = (tmp + 288) / 146097;
+
+  if (first_year == last_year) {
+    *year = first_year;
+  } else if (CalcFirstDayOfYearSinceEpoch(last_year) <= days_since_epoch_) {
+    *year = last_year;
+  } else {
+    *year = first_year;
+  }
+
+  return true;
+}
+
+int DateValue::WeekDay() const {
+  if (UNLIKELY(!IsValid())) return -1;
+  const cctz::civil_day cd = EPOCH_DATE + days_since_epoch_;
+  return static_cast<int>(cctz::get_weekday(cd));
+}
+
+DateValue DateValue::AddDays(int days) const {
+  if (UNLIKELY(!IsValid())) return DateValue();
+  return DateValue(days_since_epoch_ + days);
+}
+
 bool DateValue::ToDaysSinceEpoch(int32_t* days) const {
   DCHECK(days != nullptr);
   if (UNLIKELY(!IsValid())) return false;
diff --git a/be/src/runtime/date-value.h b/be/src/runtime/date-value.h
index e927d7d..606eb7a 100644
--- a/be/src/runtime/date-value.h
+++ b/be/src/runtime/date-value.h
@@ -82,6 +82,20 @@ class DateValue {
   /// Otherwise, return false.
   bool ToYearMonthDay(int* year, int* month, int* day) const WARN_UNUSED_RESULT;
 
+  /// If this DateValue instance is valid, convert it to year and return true. Result is
+  /// placed in 'year'.
+  /// Otherwise, return false.
+  bool ToYear(int* year) const WARN_UNUSED_RESULT;
+
+  /// If DateValue instance is valid, returns day-of-week in [0, 6]; 0 = Monday and
+  /// 6 = Sunday.
+  /// Otherwise, return -1.
+  int WeekDay() const;
+
+  /// If this DateValue instance valid, add 'days' to it and return the result.
+  /// Otherwise, return an invalid DateValue instance.
+  DateValue AddDays(int days) const;
+
   /// If this DateValue instance is valid, convert it to the number of days since epoch
   /// and return true. Result is placed in 'days'.
   /// Otherwise, return false.
diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py
index e702db7..44ee32e 100644
--- a/common/function-registry/impala_functions.py
+++ b/common/function-registry/impala_functions.py
@@ -81,21 +81,33 @@ visible_functions = [
   [['is_inf'], 'BOOLEAN', ['DOUBLE'],
    '_ZN6impala11UdfBuiltins5IsInfEPN10impala_udf15FunctionContextERKNS1_9DoubleValE'],
   [['trunc'], 'TIMESTAMP', ['TIMESTAMP', 'STRING'],
-   '_ZN6impala11UdfBuiltins5TruncEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
-   '_ZN6impala11UdfBuiltins12TruncPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins10TruncCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins17TruncForTimestampEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins24TruncForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins22TruncForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['trunc'], 'DATE', ['DATE', 'STRING'],
+   '_ZN6impala11UdfBuiltins12TruncForDateEPN10impala_udf15FunctionContextERKNS1_7DateValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins19TruncForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins17TruncForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   # Don't add an entry for EXTRACT(STRING, TIMESTAMP). STRINGs may be used to represent
   # TIMESTAMPs meaning EXTRACT(STRING, STRING) is valid. If EXTRACT(STRING, TIMESTAMP)
   # is added, it takes precedence over the existing EXTRACT(TIMESTAMP, STRING)
   # which could break users.
   [['extract'], 'BIGINT', ['TIMESTAMP', 'STRING'],
-   '_ZN6impala11UdfBuiltins7ExtractEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
-   '_ZN6impala11UdfBuiltins21SwappedExtractPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins12ExtractCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins19ExtractForTimestampEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins26ExtractForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins24ExtractForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   [['date_part'], 'BIGINT', ['STRING', 'TIMESTAMP'],
-   '_ZN6impala11UdfBuiltins7ExtractEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
-   '_ZN6impala11UdfBuiltins14ExtractPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins12ExtractCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins20DatePartForTimestampEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
+   '_ZN6impala11UdfBuiltins27DatePartForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins25DatePartForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['extract'], 'BIGINT', ['DATE', 'STRING'],
+   '_ZN6impala11UdfBuiltins14ExtractForDateEPN10impala_udf15FunctionContextERKNS1_7DateValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins21ExtractForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins19ExtractForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['date_part'], 'BIGINT', ['STRING', 'DATE'],
+   '_ZN6impala11UdfBuiltins15DatePartForDateEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_7DateValE',
+   '_ZN6impala11UdfBuiltins22DatePartForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins20DatePartForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
 
   [['madlib_encode_vector'], 'STRING', ['STRING'],
     '_ZN6impala11UdfBuiltins12EncodeVectorEPN10impala_udf15FunctionContextERKNS1_9StringValE'],
@@ -127,9 +139,9 @@ visible_functions = [
   [['to_date'], 'STRING', ['TIMESTAMP'], '_ZN6impala18TimestampFunctions6ToDateEPN10impala_udf15FunctionContextERKNS1_12TimestampValE'],
   [['dayname'], 'STRING', ['TIMESTAMP'], '_ZN6impala18TimestampFunctions11LongDayNameEPN10impala_udf15FunctionContextERKNS1_12TimestampValE'],
   [['date_trunc'], 'TIMESTAMP', ['STRING', 'TIMESTAMP'],
-   '_ZN6impala11UdfBuiltins9DateTruncEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
-   '_ZN6impala11UdfBuiltins16DateTruncPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins14DateTruncCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins21DateTruncForTimestampEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
+   '_ZN6impala11UdfBuiltins28DateTruncForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins26DateTruncForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'],
       '_ZN6impala18TimestampFunctions6AddSubILb1EN10impala_udf6IntValEN5boost9date_time14years_durationINS4_9gregorian21greg_durations_configEEELb0EEENS2_12TimestampValEPNS2_15FunctionContextERKSA_RKT0_'],
   [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'],
@@ -256,6 +268,12 @@ visible_functions = [
    '_ZN6impala18TimestampFunctions22UnixAndFromUnixPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
    '_ZN6impala18TimestampFunctions20UnixAndFromUnixCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
 
+  # Date functions
+  [['date_trunc'], 'DATE', ['STRING', 'DATE'],
+   '_ZN6impala11UdfBuiltins16DateTruncForDateEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_7DateValE',
+   '_ZN6impala11UdfBuiltins23DateTruncForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins21DateTruncForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+
   # Math builtin functions
   [['pi'], 'DOUBLE', [], 'impala::MathFunctions::Pi'],
   [['e'], 'DOUBLE', [], 'impala::MathFunctions::E'],
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 d732682..3d850aa 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
@@ -36,15 +36,25 @@ public class ExtractFromExpr extends FunctionCallExpr {
 
   // Behaves like an immutable linked hash set containing the TExtractFields in the same
   // order as declared.
-  private static final Set<String> EXTRACT_FIELDS;
+  private static final Set<String> TIMESTAMP_EXTRACT_FIELDS;
+  private static final Set<String> DATE_EXTRACT_FIELDS;
   static {
-    ImmutableSet.Builder<String> builder = new ImmutableSet.Builder<String>();
+    ImmutableSet.Builder<String> timestamp_builder = new ImmutableSet.Builder<String>();
+    ImmutableSet.Builder<String> date_builder = new ImmutableSet.Builder<String>();
     for (TExtractField extractField: TExtractField.values()) {
       if (extractField != TExtractField.INVALID_FIELD) {
-        builder.add(extractField.name());
+        timestamp_builder.add(extractField.name());
+
+        if (extractField != TExtractField.HOUR && extractField != TExtractField.MINUTE
+            && extractField != TExtractField.SECOND
+            && extractField != TExtractField.MILLISECOND
+            && extractField != TExtractField.EPOCH) {
+          date_builder.add(extractField.name());
+        }
       }
     }
-    EXTRACT_FIELDS = builder.build();
+    TIMESTAMP_EXTRACT_FIELDS = timestamp_builder.build();
+    DATE_EXTRACT_FIELDS = date_builder.build();
   }
 
   public ExtractFromExpr(FunctionName fnName, String extractFieldIdent, Expr e) {
@@ -83,10 +93,18 @@ public class ExtractFromExpr extends FunctionCallExpr {
     String extractFieldIdent =
         ((StringLiteral)children_.get(1)).getValueWithOriginalEscapes();
     Preconditions.checkNotNull(extractFieldIdent);
-    if (!EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase())) {
+
+    boolean isDate = children_.get(0).getType().isDate();
+    boolean isExtractFieldValid = isDate ?
+        DATE_EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase()) :
+        TIMESTAMP_EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase());
+
+    if (!isExtractFieldValid) {
+      String validExtractFields = Joiner.on(", ").join(
+          isDate ? DATE_EXTRACT_FIELDS : TIMESTAMP_EXTRACT_FIELDS);
       throw new AnalysisException("Time unit '" + extractFieldIdent + "' in expression '"
           + toSql() + "' is invalid. Expected one of "
-          + Joiner.on(", ").join(EXTRACT_FIELDS) + ".");
+          + validExtractFields + ".");
     }
   }
 
@@ -94,7 +112,7 @@ public class ExtractFromExpr extends FunctionCallExpr {
   protected String getFunctionNotFoundError(Type[] argTypes) {
     Expr e = children_.get(0);
     return "Expression '" + e.toSql() + "' in '" + toSql() + "' has a return type of "
-          + e.getType().toSql() + " but a TIMESTAMP is required.";
+          + e.getType().toSql() + " but a TIMESTAMP or DATE is required.";
   }
 
   @Override
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 62cb502..2bb4238 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -1866,12 +1866,21 @@ public class AnalyzeExprsTest extends AnalyzerTest {
 
     // Special cases for FROM in function call
     AnalyzesOk("select extract(year from now())");
+    AnalyzesOk("select extract(year from cast(now() as date))");
+    AnalyzesOk("select extract(year from date_col) from functional.date_tbl");
+    AnalyzesOk("select extract(hour from now())");
+    AnalysisError("select extract(hour from cast(now() as date))",
+        "Time unit 'hour' in expression 'EXTRACT(hour FROM CAST(now() AS DATE))' is " +
+        "invalid. Expected one of YEAR, QUARTER, MONTH, DAY.");
     AnalysisError("select extract(foo from now())",
         "Time unit 'foo' in expression 'EXTRACT(foo FROM now())' is invalid. Expected " +
         "one of YEAR, QUARTER, MONTH, DAY, HOUR, MINUTE, SECOND, MILLISECOND, EPOCH.");
+    AnalysisError("select extract(foo from date_col) from functional.date_tbl",
+        "Time unit 'foo' in expression 'EXTRACT(foo FROM date_col)' is " +
+        "invalid. Expected one of YEAR, QUARTER, MONTH, DAY.");
     AnalysisError("select extract(year from 0)",
         "Expression '0' in 'EXTRACT(year FROM 0)' has a return type of TINYINT but a " +
-        "TIMESTAMP is required.");
+        "TIMESTAMP or DATE is required.");
     AnalysisError("select functional.extract(year from now())",
         "Function functional.extract conflicts with the EXTRACT builtin");
     AnalysisError("select date_part(year from now())",
diff --git a/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test b/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
index f38287b..a8482b1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
@@ -178,5 +178,5 @@ BIGINT
 select int_col from functional.alltypes
 where year=date_part('yyyyMMMdd hh:mm:ss', current_timestamp());
 ---- CATCH
-InternalException: invalid extract field: yyyyMMMdd hh:mm:ss
+InternalException: Invalid Date Part Field: yyyyMMMdd hh:mm:ss
 ====