You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2017/05/11 21:28:20 UTC

[2/2] incubator-impala git commit: IMPALA-5137: Support Kudu UNIXTIME_MICROS as Impala TIMESTAMP

IMPALA-5137: Support Kudu UNIXTIME_MICROS as Impala TIMESTAMP

Adds Impala support for TIMESTAMP types stored in Kudu.

Impala stores TIMESTAMP values in 96-bits and has nanosecond
precision. Kudu's timestamp is a 64-bit microsecond delta
from the Unix epoch (called UNIXTIME_MICROS), so a conversion
is necessary.

When writing to Kudu, TIMESTAMP values in nanoseconds are
averaged to the nearest microsecond.

When reading from Kudu, the KuduScanner returns
UNIXTIME_MICROS with 8bytes of padding so Impala can convert
the value to a TimestampValue in-line and copy the entire
row.

Testing:
Updated the functional_kudu schema to use TIMESTAMPs instead
of converting to STRING, so this provides some decent
coverage. Some BE tests were added, and some EE tests as
well.

TODO: Support pushing down TIMESTAMP predicates
TODO: Support TIMESTAMPs in range partitioning expressions

Change-Id: Iae6ccfffb79118a9036fb2227dba3a55356c896d
Reviewed-on: http://gerrit.cloudera.org:8080/6526
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: a16a0fa84d77f96e428b278f8cc37ebd7a49899f
Parents: cb37be8
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Tue Mar 28 19:05:03 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu May 11 20:55:51 2017 +0000

----------------------------------------------------------------------
 .../benchmarks/convert-timestamp-benchmark.cc   |   2 +-
 be/src/benchmarks/parse-timestamp-benchmark.cc  |   8 +-
 be/src/exec/data-source-scan-node.cc            |   2 +-
 be/src/exec/kudu-scanner.cc                     |  45 ++++-
 be/src/exec/kudu-scanner.h                      |   4 +
 be/src/exec/kudu-table-sink.cc                  |   2 +-
 be/src/exec/kudu-util.cc                        |  26 ++-
 be/src/exec/kudu-util.h                         |  30 +--
 be/src/exec/parquet-column-readers.cc           |   2 +-
 be/src/exec/text-converter.inline.h             |   2 +-
 be/src/exprs/aggregate-functions-ir.cc          |  10 +-
 be/src/exprs/cast-functions-ir.cc               |  24 ++-
 be/src/exprs/decimal-operators-ir.cc            |   6 +-
 be/src/exprs/expr-test.cc                       | 129 ++++++++-----
 be/src/exprs/expr-value.h                       |   1 -
 be/src/exprs/kudu-partition-expr.cc             |   4 +-
 be/src/exprs/literal.cc                         |   2 +-
 be/src/exprs/timestamp-functions-ir.cc          |  16 +-
 be/src/exprs/timestamp-functions.cc             |   4 +-
 be/src/runtime/raw-value-test.cc                |   2 +-
 be/src/runtime/runtime-state.cc                 |   5 +-
 be/src/runtime/timestamp-test.cc                | 148 ++++++++++----
 be/src/runtime/timestamp-value.cc               |  35 ++--
 be/src/runtime/timestamp-value.h                |  79 ++++----
 be/src/runtime/timestamp-value.inline.h         | 101 ++++++++++
 be/src/service/client-request-state.cc          |   4 +-
 be/src/service/impala-http-handler.cc           |  16 +-
 be/src/service/impala-server.cc                 |  15 +-
 be/src/statestore/statestore-subscriber.cc      |   2 +-
 be/src/util/bloom-filter.h                      |   1 +
 be/src/util/dict-test.cc                        |   6 +-
 be/src/util/promise.h                           |   3 +-
 common/thrift/generate_error_codes.py           |   4 +
 .../java/org/apache/impala/util/KuduUtil.java   |   3 +-
 .../apache/impala/analysis/AnalyzeDDLTest.java  |   7 +-
 .../functional/functional_schema_template.sql   |  24 +--
 .../kudu-overflow-ts-abort-on-error.test        |   6 +
 .../queries/QueryTest/kudu-overflow-ts.test     |  11 ++
 .../queries/QueryTest/kudu_describe.test        |   2 +-
 .../queries/QueryTest/kudu_insert.test          | 191 ++++++++++---------
 tests/query_test/test_kudu.py                   |  46 +++--
 tests/query_test/test_queries.py                |   8 +-
 42 files changed, 702 insertions(+), 336 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/benchmarks/convert-timestamp-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/convert-timestamp-benchmark.cc b/be/src/benchmarks/convert-timestamp-benchmark.cc
index 6ba7183..8a5cd3e 100644
--- a/be/src/benchmarks/convert-timestamp-benchmark.cc
+++ b/be/src/benchmarks/convert-timestamp-benchmark.cc
@@ -77,7 +77,7 @@ void AddTestDataDateTimes(TestData* data, int n, const string& startstr) {
     stringstream ss;
     ss << to_simple_string(start);
     string ts = ss.str();
-    data->data.push_back(TimestampValue(ts.c_str(), ts.size(), dt_ctx));
+    data->data.push_back(TimestampValue::Parse(ts.c_str(), ts.size(), dt_ctx));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/benchmarks/parse-timestamp-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/parse-timestamp-benchmark.cc b/be/src/benchmarks/parse-timestamp-benchmark.cc
index fc9c8d8..7aad0e6 100644
--- a/be/src/benchmarks/parse-timestamp-benchmark.cc
+++ b/be/src/benchmarks/parse-timestamp-benchmark.cc
@@ -128,7 +128,7 @@ void TestImpalaDate(int batch_size, void* d) {
   for (int i = 0; i < batch_size; ++i) {
     int n = data->data.size();
     for (int j = 0; j < n; ++j) {
-      data->result[j] = TimestampValue(data->data[j].ptr, data->data[j].len);
+      data->result[j] = TimestampValue::Parse(data->data[j].ptr, data->data[j].len);
     }
   }
 }
@@ -170,7 +170,8 @@ void TestImpalaTimestamp(int batch_size, void* d) {
   for (int i = 0; i < batch_size; ++i) {
     int n = data->data.size();
     for (int j = 0; j < n; ++j) {
-      data->result[j] = TimestampValue(data->data[j].ptr, data->data[j].len, dt_ctx);
+      data->result[j] = TimestampValue::Parse(data->data[j].ptr, data->data[j].len,
+          dt_ctx);
     }
   }
 }
@@ -180,7 +181,8 @@ void TestImpalaTZTimestamp(int batch_size, void* d) {
   for (int i = 0; i < batch_size; ++i) {
     int n = data->data.size();
     for (int j = 0; j < n; ++j) {
-      data->result[j] = TimestampValue(data->data[j].ptr, data->data[j].len, dt_ctx_tz);
+      data->result[j] = TimestampValue::Parse(data->data[j].ptr, data->data[j].len,
+          dt_ctx_tz);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/data-source-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-source-scan-node.cc b/be/src/exec/data-source-scan-node.cc
index 3198a73..c0a9287 100644
--- a/be/src/exec/data-source-scan-node.cc
+++ b/be/src/exec/data-source-scan-node.cc
@@ -281,7 +281,7 @@ Status DataSourceScanNode::MaterializeNextRow(MemPool* tuple_pool, Tuple* tuple)
         const string& val = col.binary_vals[val_idx];
         if (val.size() != TIMESTAMP_SIZE) return Status(ERROR_INVALID_TIMESTAMP);
         const uint8_t* bytes = reinterpret_cast<const uint8_t*>(val.data());
-        *reinterpret_cast<TimestampValue*>(slot) = TimestampValue(
+        *reinterpret_cast<TimestampValue*>(slot) = TimestampValue::FromUnixTimeNanos(
             ReadWriteUtil::GetInt<uint64_t>(bytes),
             ReadWriteUtil::GetInt<uint32_t>(bytes + sizeof(int64_t)));
         break;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index 2495a58..8ed160f 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -27,6 +27,7 @@
 #include "exec/kudu-util.h"
 #include "runtime/mem-pool.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/raw-value.h"
 #include "runtime/runtime-state.h"
 #include "runtime/row-batch.h"
 #include "runtime/string-value.h"
@@ -66,6 +67,11 @@ KuduScanner::KuduScanner(KuduScanNodeBase* scan_node, RuntimeState* state)
 }
 
 Status KuduScanner::Open() {
+  for (int i = 0; i < scan_node_->tuple_desc()->slots().size(); ++i) {
+    const SlotDescriptor* slot = scan_node_->tuple_desc()->slots()[i];
+    if (slot->type().type != TYPE_TIMESTAMP) continue;
+    timestamp_slots_.push_back(slot);
+  }
   return scan_node_->GetConjunctCtxs(&conjunct_ctxs_);
 }
 
@@ -145,6 +151,8 @@ Status KuduScanner::OpenNextScanToken(const string& scan_token)  {
       "Could not set scanner timeout");
   VLOG_ROW << "Starting KuduScanner with ReadMode=" << mode << " timeout=" <<
       FLAGS_kudu_operation_timeout_ms;
+  uint64_t row_format_flags = kudu::client::KuduScanner::PAD_UNIXTIME_MICROS_TO_16_BYTES;
+  scanner_->SetRowFormatFlags(row_format_flags);
 
   {
     SCOPED_TIMER(state_->total_storage_wait_timer());
@@ -178,13 +186,44 @@ Status KuduScanner::DecodeRowsIntoRowBatch(RowBatch* row_batch, Tuple** tuple_me
   // 'row_batch'.
   bool has_conjuncts = !conjunct_ctxs_.empty();
   int num_rows = cur_kudu_batch_.NumRows();
+
   for (int krow_idx = cur_kudu_batch_num_read_; krow_idx < num_rows; ++krow_idx) {
+    Tuple* kudu_tuple = const_cast<Tuple*>(reinterpret_cast<const Tuple*>(
+        cur_kudu_batch_.direct_data().data() +
+        (krow_idx * scan_node_->row_desc().GetRowSize())));
+    ++cur_kudu_batch_num_read_;
+
+    // Kudu tuples containing TIMESTAMP columns (UNIXTIME_MICROS in Kudu, stored as an
+    // int64) have 8 bytes of padding following the timestamp. Because this padding is
+    // provided, Impala can convert these unixtime values to Impala's TimestampValue
+    // format in place and copy the rows to Impala row batches.
+    // TODO: avoid mem copies with a Kudu mem 'release' mechanism, attaching mem to the
+    // batch.
+    // TODO: consider codegen for this per-timestamp col fixup
+    for (const SlotDescriptor* slot : timestamp_slots_) {
+      DCHECK(slot->type().type == TYPE_TIMESTAMP);
+      if (slot->is_nullable() && kudu_tuple->IsNull(slot->null_indicator_offset())) {
+        continue;
+      }
+      int64_t ts_micros = *reinterpret_cast<int64_t*>(
+          kudu_tuple->GetSlot(slot->tuple_offset()));
+      int64_t ts_seconds = ts_micros / MICROS_PER_SEC;
+      int64_t micros_part = ts_micros - (ts_seconds * MICROS_PER_SEC);
+      TimestampValue tv = TimestampValue::FromUnixTimeMicros(ts_seconds, micros_part);
+      if (tv.HasDateAndTime()) {
+        RawValue::Write(&tv, kudu_tuple, slot, NULL);
+      } else {
+        kudu_tuple->SetNull(slot->null_indicator_offset());
+        RETURN_IF_ERROR(state_->LogOrReturnError(
+            ErrorMsg::Init(TErrorCode::KUDU_TIMESTAMP_OUT_OF_RANGE,
+              scan_node_->table_->name(),
+              scan_node_->table_->schema().Column(slot->col_pos()).name())));
+      }
+    }
+
     // Evaluate the conjuncts that haven't been pushed down to Kudu. Conjunct evaluation
     // is performed directly on the Kudu tuple because its memory layout is identical to
     // Impala's. We only copy the surviving tuples to Impala's output row batch.
-    KuduScanBatch::RowPtr krow = cur_kudu_batch_.Row(krow_idx);
-    Tuple* kudu_tuple = reinterpret_cast<Tuple*>(const_cast<void*>(krow.cell(0)));
-    ++cur_kudu_batch_num_read_;
     if (has_conjuncts && !ExecNode::EvalConjuncts(&conjunct_ctxs_[0],
         conjunct_ctxs_.size(), reinterpret_cast<TupleRow*>(&kudu_tuple))) {
       continue;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/kudu-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.h b/be/src/exec/kudu-scanner.h
index 90ff68d..7a6ca76 100644
--- a/be/src/exec/kudu-scanner.h
+++ b/be/src/exec/kudu-scanner.h
@@ -101,6 +101,10 @@ class KuduScanner {
 
   /// The scanner's cloned copy of the conjuncts to apply.
   vector<ExprContext*> conjunct_ctxs_;
+
+  /// Timestamp slots in the tuple descriptor of the scan node. Used to convert Kudu
+  /// UNIXTIME_MICRO values inline.
+  vector<const SlotDescriptor*> timestamp_slots_;
 };
 
 } /// namespace impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/kudu-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.cc b/be/src/exec/kudu-table-sink.cc
index f09b832..75ca2fc 100644
--- a/be/src/exec/kudu-table-sink.cc
+++ b/be/src/exec/kudu-table-sink.cc
@@ -251,7 +251,7 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
       }
 
       PrimitiveType type = output_expr_ctxs_[j]->root()->type().type;
-      WriteKuduRowValue(write->mutable_row(), col, type, value);
+      WriteKuduValue(col, type, value, true, write->mutable_row());
     }
     if (add_row) write_ops.push_back(move(write));
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/kudu-util.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.cc b/be/src/exec/kudu-util.cc
index 59a4c81..6688b7e 100644
--- a/be/src/exec/kudu-util.cc
+++ b/be/src/exec/kudu-util.cc
@@ -27,7 +27,8 @@
 #include "common/logging.h"
 #include "common/names.h"
 #include "common/status.h"
-#include "runtime/string-value.h"
+#include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 
 using kudu::client::KuduSchema;
 using kudu::client::KuduClient;
@@ -108,8 +109,22 @@ void InitKuduLogging() {
   kudu::client::SetVerboseLogLevel(std::max(0, FLAGS_v - 1));
 }
 
-Status WriteKuduRowValue(kudu::KuduPartialRow* row, int col, PrimitiveType type,
-    const void* value, bool copy_strings) {
+Status WriteKuduTimestampValue(int col, const TimestampValue* tv,
+    kudu::KuduPartialRow* row) {
+  int64_t ts_micros;
+  bool success = tv->UtcToUnixTimeMicros(&ts_micros);
+  DCHECK(success); // If the value was invalid the slot should've been null.
+  if (UNLIKELY(!success)) {
+    return Status(TErrorCode::RUNTIME_ERROR,
+        "Invalid TimestampValue: " + tv->ToString());
+  }
+  KUDU_RETURN_IF_ERROR(row->SetUnixTimeMicros(col, ts_micros),
+      "Could not add Kudu WriteOp.");
+  return Status::OK();
+}
+
+Status WriteKuduValue(int col, PrimitiveType type, const void* value,
+    bool copy_strings, kudu::KuduPartialRow* row) {
   // TODO: codegen this to eliminate braching on type.
   switch (type) {
     case TYPE_VARCHAR:
@@ -152,11 +167,14 @@ Status WriteKuduRowValue(kudu::KuduPartialRow* row, int col, PrimitiveType type,
       KUDU_RETURN_IF_ERROR(row->SetInt64(col, *reinterpret_cast<const int64_t*>(value)),
           "Could not set Kudu row value.");
       break;
+    case TYPE_TIMESTAMP:
+      RETURN_IF_ERROR(WriteKuduTimestampValue(col,
+          reinterpret_cast<const TimestampValue*>(value), row));
+      break;
     default:
       return Status(TErrorCode::IMPALA_KUDU_TYPE_MISSING, TypeToString(type));
   }
 
   return Status::OK();
 }
-
 }  // namespace impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/kudu-util.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.h b/be/src/exec/kudu-util.h
index 27765df..bc6d539 100644
--- a/be/src/exec/kudu-util.h
+++ b/be/src/exec/kudu-util.h
@@ -21,11 +21,22 @@
 #include <kudu/client/callbacks.h>
 #include <kudu/client/client.h>
 
+#include "common/status.h"
+#include "runtime/string-value.h"
 #include "runtime/types.h"
 
 namespace impala {
 
-class Status;
+/// Takes a Kudu status and returns an impala one, if it's not OK.
+#define KUDU_RETURN_IF_ERROR(expr, prepend) \
+  do { \
+    kudu::Status _s = (expr); \
+    if (UNLIKELY(!_s.ok())) {                                      \
+      return Status(strings::Substitute("$0: $1", prepend, _s.ToString())); \
+    } \
+  } while (0)
+
+class TimestampValue;
 
 /// Returns false when running on an operating system that Kudu doesn't support. If this
 /// check fails, there is no way Kudu should be expected to work. Exposed for testing.
@@ -59,19 +70,10 @@ void LogKuduMessage(kudu::client::KuduLogSeverity severity, const char* filename
 
 /// Casts 'value' according to 'type' and writes it into 'row' at position 'col'.
 /// If 'type' is STRING or VARCHAR, 'copy_strings' determines if 'value' will be copied
-/// into memory owned by the row. If false, string data must remain valid while the row is
-/// being used.
-Status WriteKuduRowValue(kudu::KuduPartialRow* row, int col, PrimitiveType type,
-    const void* value, bool copy_strings = true);
-
-/// Takes a Kudu status and returns an impala one, if it's not OK.
-#define KUDU_RETURN_IF_ERROR(expr, prepend) \
-  do { \
-    kudu::Status _s = (expr); \
-    if (UNLIKELY(!_s.ok())) {                                      \
-      return Status(strings::Substitute("$0: $1", prepend, _s.ToString())); \
-    } \
-  } while (0)
+/// into memory owned by the row. If false, string data must remain valid while the row
+/// is being used.
+Status WriteKuduValue(int col, PrimitiveType type, const void* value,
+    bool copy_strings, kudu::KuduPartialRow* row);
 
 } /// namespace impala
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/parquet-column-readers.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-readers.cc b/be/src/exec/parquet-column-readers.cc
index f1ac031..cb7a5dd 100644
--- a/be/src/exec/parquet-column-readers.cc
+++ b/be/src/exec/parquet-column-readers.cc
@@ -614,7 +614,7 @@ inline bool ScalarColumnReader<TimestampValue, true>::NeedsConversionInline() co
 bool __attribute__((noinline)) SetTimestampConversionError(HdfsScanNodeBase* scan_node,
     RuntimeState* scanner_state, const TimestampValue* tv, const string& timezone,
     const string& detail, Status* scanner_status) {
-  ErrorMsg msg(TErrorCode::PARQUET_MR_TIMESTAMP_CONVERSION_FAILED, tv->DebugString(),
+  ErrorMsg msg(TErrorCode::PARQUET_MR_TIMESTAMP_CONVERSION_FAILED, tv->ToString(),
       timezone, scan_node->hdfs_table()->fully_qualified_name());
   if (!detail.empty()) msg.AddDetail(detail);
   Status status = scanner_state->LogOrReturnError(msg);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exec/text-converter.inline.h
----------------------------------------------------------------------
diff --git a/be/src/exec/text-converter.inline.h b/be/src/exec/text-converter.inline.h
index 5ee8221..d8dea91 100644
--- a/be/src/exec/text-converter.inline.h
+++ b/be/src/exec/text-converter.inline.h
@@ -130,7 +130,7 @@ inline bool TextConverter::WriteSlot(const SlotDescriptor* slot_desc, Tuple* tup
       break;
     case TYPE_TIMESTAMP: {
       TimestampValue* ts_slot = reinterpret_cast<TimestampValue*>(slot);
-      *ts_slot = TimestampValue(data, len);
+      *ts_slot = TimestampValue::Parse(data, len);
       if (!ts_slot->HasDateOrTime()) {
         parse_result = StringParser::PARSE_FAILURE;
       }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/aggregate-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/aggregate-functions-ir.cc b/be/src/exprs/aggregate-functions-ir.cc
index 611945a..74badf6 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -32,6 +32,7 @@
 #include "runtime/runtime-state.h"
 #include "runtime/string-value.inline.h"
 #include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "exprs/anyval-util.h"
 #include "exprs/expr.h"
 #include "exprs/hll-bias.h"
@@ -324,7 +325,7 @@ void AggregateFunctions::TimestampAvgUpdate(FunctionContext* ctx,
   DCHECK(dst->ptr != NULL);
   DCHECK_EQ(sizeof(AvgState), dst->len);
   AvgState* avg = reinterpret_cast<AvgState*>(dst->ptr);
-  TimestampValue tm_src = TimestampValue::FromTimestampVal(src);
+  const TimestampValue& tm_src = TimestampValue::FromTimestampVal(src);
   double val;
   if (tm_src.ToSubsecondUnixTime(&val)) {
     avg->sum += val;
@@ -338,7 +339,7 @@ void AggregateFunctions::TimestampAvgRemove(FunctionContext* ctx,
   DCHECK(dst->ptr != NULL);
   DCHECK_EQ(sizeof(AvgState), dst->len);
   AvgState* avg = reinterpret_cast<AvgState*>(dst->ptr);
-  TimestampValue tm_src = TimestampValue::FromTimestampVal(src);
+  const TimestampValue& tm_src = TimestampValue::FromTimestampVal(src);
   double val;
   if (tm_src.ToSubsecondUnixTime(&val)) {
     avg->sum -= val;
@@ -351,7 +352,8 @@ TimestampVal AggregateFunctions::TimestampAvgGetValue(FunctionContext* ctx,
     const StringVal& src) {
   AvgState* val_struct = reinterpret_cast<AvgState*>(src.ptr);
   if (val_struct->count == 0) return TimestampVal::null();
-  TimestampValue tv(val_struct->sum / val_struct->count);
+  const TimestampValue& tv = TimestampValue::FromSubsecondUnixTime(
+      val_struct->sum / val_struct->count);
   if (tv.HasDate()) {
     TimestampVal result;
     tv.ToTimestampVal(&result);
@@ -1275,7 +1277,7 @@ void PrintSample(const ReservoirSample<DecimalVal>& v, ostream* os) {
 
 template <>
 void PrintSample(const ReservoirSample<TimestampVal>& v, ostream* os) {
-  *os << TimestampValue::FromTimestampVal(v.val).DebugString();
+  *os << TimestampValue::FromTimestampVal(v.val).ToString();
 }
 
 template <typename T>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/cast-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/cast-functions-ir.cc b/be/src/exprs/cast-functions-ir.cc
index 9fa8268..518dd5a 100644
--- a/be/src/exprs/cast-functions-ir.cc
+++ b/be/src/exprs/cast-functions-ir.cc
@@ -26,6 +26,7 @@
 #include "runtime/runtime-state.h"
 #include "runtime/string-value.inline.h"
 #include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "util/string-parser.h"
 #include "string-functions.h"
 
@@ -229,11 +230,24 @@ CAST_FROM_TIMESTAMP(BigIntVal);
 CAST_FROM_SUBSECOND_TIMESTAMP(FloatVal);
 CAST_FROM_SUBSECOND_TIMESTAMP(DoubleVal);
 
+#define CAST_TO_SUBSECOND_TIMESTAMP(from_type) \
+  TimestampVal CastFunctions::CastToTimestampVal(FunctionContext* ctx, \
+                                                 const from_type& val) { \
+    if (val.is_null) return TimestampVal::null(); \
+    TimestampValue timestamp_value = TimestampValue::FromSubsecondUnixTime(val.val); \
+    TimestampVal result; \
+    timestamp_value.ToTimestampVal(&result); \
+    return result; \
+  }
+
+CAST_TO_SUBSECOND_TIMESTAMP(FloatVal);
+CAST_TO_SUBSECOND_TIMESTAMP(DoubleVal);
+
 #define CAST_TO_TIMESTAMP(from_type) \
   TimestampVal CastFunctions::CastToTimestampVal(FunctionContext* ctx, \
                                                  const from_type& val) { \
     if (val.is_null) return TimestampVal::null(); \
-    TimestampValue timestamp_value(val.val); \
+    TimestampValue timestamp_value = TimestampValue::FromUnixTime(val.val); \
     TimestampVal result; \
     timestamp_value.ToTimestampVal(&result); \
     return result; \
@@ -244,15 +258,13 @@ CAST_TO_TIMESTAMP(TinyIntVal);
 CAST_TO_TIMESTAMP(SmallIntVal);
 CAST_TO_TIMESTAMP(IntVal);
 CAST_TO_TIMESTAMP(BigIntVal);
-CAST_TO_TIMESTAMP(FloatVal);
-CAST_TO_TIMESTAMP(DoubleVal);
 
 TimestampVal CastFunctions::CastToTimestampVal(FunctionContext* ctx,
-                                               const StringVal& val) {
+    const StringVal& val) {
   if (val.is_null) return TimestampVal::null();
-  TimestampValue timestamp_value(reinterpret_cast<char*>(val.ptr), val.len);
+  TimestampValue tv = TimestampValue::Parse(reinterpret_cast<char*>(val.ptr), val.len);
   // Return null if 'val' did not parse
   TimestampVal result;
-  timestamp_value.ToTimestampVal(&result);
+  tv.ToTimestampVal(&result);
   return result;
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/decimal-operators-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/decimal-operators-ir.cc b/be/src/exprs/decimal-operators-ir.cc
index e502dda..785b1b3 100644
--- a/be/src/exprs/decimal-operators-ir.cc
+++ b/be/src/exprs/decimal-operators-ir.cc
@@ -616,7 +616,7 @@ TimestampVal DecimalOperators::CastToTimestampVal(
       int32_t seconds = dv.whole_part(scale);
       int32_t nanoseconds = ConvertToNanoseconds(
           dv.fractional_part(scale), scale);
-      TimestampValue tv(seconds, nanoseconds);
+      TimestampValue tv = TimestampValue::FromUnixTimeNanos(seconds, nanoseconds);
       tv.ToTimestampVal(&result);
       break;
     }
@@ -625,7 +625,7 @@ TimestampVal DecimalOperators::CastToTimestampVal(
       int64_t seconds = dv.whole_part(scale);
       int64_t nanoseconds = ConvertToNanoseconds(
           dv.fractional_part(scale), scale);
-      TimestampValue tv(seconds, nanoseconds);
+      TimestampValue tv = TimestampValue::FromUnixTimeNanos(seconds, nanoseconds);
       tv.ToTimestampVal(&result);
       break;
     }
@@ -639,7 +639,7 @@ TimestampVal DecimalOperators::CastToTimestampVal(
       }
       int128_t nanoseconds = ConvertToNanoseconds(
           dv.fractional_part(scale), scale);
-      TimestampValue tv(seconds, nanoseconds);
+      TimestampValue tv = TimestampValue::FromUnixTimeNanos(seconds, nanoseconds);
       tv.ToTimestampVal(&result);
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index a775e7b..5ff26a1 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -211,7 +211,7 @@ class ExprTest : public testing::Test {
     default_decimal_str_ = "1.23";
     default_bool_val_ = false;
     default_string_val_ = "abc";
-    default_timestamp_val_ = TimestampValue(1293872461);
+    default_timestamp_val_ = TimestampValue::FromUnixTime(1293872461);
     default_type_strs_[TYPE_TINYINT] =
         lexical_cast<string>(min_int_values_[TYPE_TINYINT]);
     default_type_strs_[TYPE_SMALLINT] =
@@ -279,67 +279,67 @@ class ExprTest : public testing::Test {
   void TestNextDayFunction() {
     // Sequential test cases
     TestTimestampValue("next_day('2016-05-01','Sunday')",
-      TimestampValue("2016-05-08 00:00:00", 19));
+      TimestampValue::Parse("2016-05-08 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Monday')",
-      TimestampValue("2016-05-02 00:00:00", 19));
+      TimestampValue::Parse("2016-05-02 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Tuesday')",
-      TimestampValue("2016-05-03 00:00:00", 19));
+      TimestampValue::Parse("2016-05-03 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Wednesday')",
-      TimestampValue("2016-05-04 00:00:00", 19));
+      TimestampValue::Parse("2016-05-04 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Thursday')",
-      TimestampValue("2016-05-05 00:00:00", 19));
+      TimestampValue::Parse("2016-05-05 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Friday')",
-      TimestampValue("2016-05-06 00:00:00", 19));
+      TimestampValue::Parse("2016-05-06 00:00:00", 19));
     TestTimestampValue("next_day('2016-05-01','Saturday')",
-      TimestampValue("2016-05-07 00:00:00", 19));
+      TimestampValue::Parse("2016-05-07 00:00:00", 19));
 
     // Random test cases
     TestTimestampValue("next_day('1910-01-18','SunDay')",
-      TimestampValue("1910-01-23 00:00:00", 19));
+      TimestampValue::Parse("1910-01-23 00:00:00", 19));
     TestTimestampValue("next_day('1916-06-05', 'SUN')",
-      TimestampValue("1916-06-11 00:00:00", 19));
+      TimestampValue::Parse("1916-06-11 00:00:00", 19));
     TestTimestampValue("next_day('1932-11-08','monday')",
-      TimestampValue("1932-11-14 00:00:00", 19));
+      TimestampValue::Parse("1932-11-14 00:00:00", 19));
     TestTimestampValue("next_day('1933-09-11','Mon')",
-      TimestampValue("1933-09-18 00:00:00", 19));
+      TimestampValue::Parse("1933-09-18 00:00:00", 19));
     TestTimestampValue("next_day('1934-03-21','TUeSday')",
-      TimestampValue("1934-03-27 00:00:00", 19));
+      TimestampValue::Parse("1934-03-27 00:00:00", 19));
     TestTimestampValue("next_day('1954-02-25','tuE')",
-      TimestampValue("1954-03-02 00:00:00", 19));
+      TimestampValue::Parse("1954-03-02 00:00:00", 19));
     TestTimestampValue("next_day('1965-04-18','WeDneSdaY')",
-      TimestampValue("1965-04-21 00:00:00", 19));
+      TimestampValue::Parse("1965-04-21 00:00:00", 19));
     TestTimestampValue("next_day('1966-08-29','wed')",
-      TimestampValue("1966-08-31 00:00:00", 19));
+      TimestampValue::Parse("1966-08-31 00:00:00", 19));
     TestTimestampValue("next_day('1968-07-23','tHurSday')",
-      TimestampValue("1968-07-25 00:00:00", 19));
+      TimestampValue::Parse("1968-07-25 00:00:00", 19));
     TestTimestampValue("next_day('1969-05-28','thu')",
-      TimestampValue("1969-05-29 00:00:00", 19));
+      TimestampValue::Parse("1969-05-29 00:00:00", 19));
     TestTimestampValue("next_day('1989-10-12','fRIDay')",
-      TimestampValue("1989-10-13 00:00:00", 19));
+      TimestampValue::Parse("1989-10-13 00:00:00", 19));
     TestTimestampValue("next_day('1973-10-02','frI')",
-      TimestampValue("1973-10-05 00:00:00", 19));
+      TimestampValue::Parse("1973-10-05 00:00:00", 19));
     TestTimestampValue("next_day('2000-02-29','saTUrDaY')",
-      TimestampValue("2000-03-04 00:00:00", 19));
+      TimestampValue::Parse("2000-03-04 00:00:00", 19));
     TestTimestampValue("next_day('2013-04-12','sat')",
-      TimestampValue("2013-04-13 00:00:00", 19));
+      TimestampValue::Parse("2013-04-13 00:00:00", 19));
     TestTimestampValue("next_day('2013-12-25','Saturday')",
-      TimestampValue("2013-12-28 00:00:00", 19));
+      TimestampValue::Parse("2013-12-28 00:00:00", 19));
 
     // Explicit timestamp conversion tests
     TestTimestampValue("next_day(to_timestamp('12-27-2008', 'MM-dd-yyyy'), 'moN')",
-      TimestampValue("2008-12-29 00:00:00", 19));
+      TimestampValue::Parse("2008-12-29 00:00:00", 19));
     TestTimestampValue("next_day(to_timestamp('2007-20-10 11:22', 'yyyy-dd-MM HH:mm'),\
-      'TUeSdaY')", TimestampValue("2007-10-23 11:22:00", 19));
+      'TUeSdaY')", TimestampValue::Parse("2007-10-23 11:22:00", 19));
     TestTimestampValue("next_day(to_timestamp('18-11-2070 09:12', 'dd-MM-yyyy HH:mm'),\
-      'WeDneSdaY')", TimestampValue("2070-11-19 09:12:00", 19));
+      'WeDneSdaY')", TimestampValue::Parse("2070-11-19 09:12:00", 19));
     TestTimestampValue("next_day(to_timestamp('12-1900-05', 'dd-yyyy-MM'), 'tHurSday')",
-      TimestampValue("1900-05-17 00:00:00", 19));
+      TimestampValue::Parse("1900-05-17 00:00:00", 19));
     TestTimestampValue("next_day(to_timestamp('08-1987-21', 'MM-yyyy-dd'), 'FRIDAY')",
-      TimestampValue("1987-08-28 00:00:00", 19));
+      TimestampValue::Parse("1987-08-28 00:00:00", 19));
     TestTimestampValue("next_day(to_timestamp('02-04-2001', 'dd-MM-yyyy'), 'SAT')",
-      TimestampValue("2001-04-07 00:00:00", 19));
+      TimestampValue::Parse("2001-04-07 00:00:00", 19));
     TestTimestampValue("next_day(to_timestamp('1970-01-31 00:00:00',\
-      'yyyy-MM-dd HH:mm:ss'), 'SunDay')", TimestampValue("1970-02-01 00:00:00", 19));
+      'yyyy-MM-dd HH:mm:ss'), 'SunDay')", TimestampValue::Parse("1970-02-01 00:00:00", 19));
 
     // Invalid input: unacceptable date parameter
     TestIsNull("next_day('12202010','Saturday')", TYPE_TIMESTAMP);
@@ -955,6 +955,10 @@ class ExprTest : public testing::Test {
     // Check factorial function exists as alias
     TestValue("factorial(3!)", TYPE_BIGINT, 720);
   }
+
+  template<typename T>
+  TimestampValue CreateTestTimestamp(T val);
+
   // Test casting stmt to all types.  Expected result is val.
   template<typename T>
   void TestCast(const string& stmt, T val, bool timestamp_out_of_range = false) {
@@ -969,13 +973,38 @@ class ExprTest : public testing::Test {
     TestValue("cast(" + stmt + " as real)", TYPE_DOUBLE, static_cast<double>(val));
     TestStringValue("cast(" + stmt + " as string)", lexical_cast<string>(val));
     if (!timestamp_out_of_range) {
-      TestTimestampValue("cast(" + stmt + " as timestamp)", TimestampValue(val));
+      TestTimestampValue("cast(" + stmt + " as timestamp)", CreateTestTimestamp(val));
     } else {
       TestIsNull("cast(" + stmt + " as timestamp)", TYPE_TIMESTAMP);
     }
   }
 };
 
+template<>
+TimestampValue ExprTest::CreateTestTimestamp(const string& val) {
+  return TimestampValue::Parse(val);
+}
+
+template<>
+TimestampValue ExprTest::CreateTestTimestamp(float val) {
+  return TimestampValue::FromSubsecondUnixTime(val);
+}
+
+template<>
+TimestampValue ExprTest::CreateTestTimestamp(double val) {
+  return TimestampValue::FromSubsecondUnixTime(val);
+}
+
+template<>
+TimestampValue ExprTest::CreateTestTimestamp(int val) {
+  return TimestampValue::FromUnixTime(val);
+}
+
+template<>
+TimestampValue ExprTest::CreateTestTimestamp(int64_t val) {
+  return TimestampValue::FromUnixTime(val);
+}
+
 // Test casting 'stmt' to each of the native types.  The result should be 'val'
 // 'stmt' is a partial stmt that could be of any valid type.
 template<>
@@ -1035,7 +1064,7 @@ int64_t ExprTest::ConvertValue<int64_t>(const string& value) {
 
 template <>
 TimestampValue ExprTest::ConvertValue<TimestampValue>(const string& value) {
-  return TimestampValue(&value[0], value.size());
+  return TimestampValue::Parse(&value[0], value.size());
 }
 
 // We can't put this into TestValue() because GTest can't resolve
@@ -2180,24 +2209,24 @@ TEST_F(ExprTest, CastExprs) {
 
   // IMPALA-3163: Test precise conversion from Decimal to Timestamp.
   TestTimestampValue("cast(cast(1457473016.1230 as decimal(17,4)) as timestamp)",
-      TimestampValue("2016-03-08 21:36:56.123000000", 29));
+      TimestampValue::Parse("2016-03-08 21:36:56.123000000", 29));
   // 32 bit Decimal.
   TestTimestampValue("cast(cast(123.45 as decimal(9,2)) as timestamp)",
-      TimestampValue("1970-01-01 00:02:03.450000000", 29));
+      TimestampValue::Parse("1970-01-01 00:02:03.450000000", 29));
   // 64 bit Decimal.
   TestTimestampValue("cast(cast(123.45 as decimal(18,2)) as timestamp)",
-      TimestampValue("1970-01-01 00:02:03.450000000", 29));
+      TimestampValue::Parse("1970-01-01 00:02:03.450000000", 29));
   TestTimestampValue("cast(cast(253402300799.99 as decimal(18, 2)) as timestamp)",
-      TimestampValue("9999-12-31 23:59:59.990000000", 29));
+      TimestampValue::Parse("9999-12-31 23:59:59.990000000", 29));
   TestIsNull("cast(cast(260000000000.00 as decimal(18, 2)) as timestamp)",
       TYPE_TIMESTAMP);
   // 128 bit Decimal.
   TestTimestampValue("cast(cast(123.45 as decimal(38,2)) as timestamp)",
-      TimestampValue("1970-01-01 00:02:03.450000000", 29));
+      TimestampValue::Parse("1970-01-01 00:02:03.450000000", 29));
   TestTimestampValue("cast(cast(253402300799.99 as decimal(38, 2)) as timestamp)",
-      TimestampValue("9999-12-31 23:59:59.990000000", 29));
+      TimestampValue::Parse("9999-12-31 23:59:59.990000000", 29));
   TestTimestampValue("cast(cast(253402300799.99 as decimal(38, 26)) as timestamp)",
-      TimestampValue("9999-12-31 23:59:59.990000000", 29));
+      TimestampValue::Parse("9999-12-31 23:59:59.990000000", 29));
   TestIsNull("cast(cast(260000000000.00 as decimal(38, 2)) as timestamp)",
       TYPE_TIMESTAMP);
   // numeric_limits<int64_t>::max()
@@ -2230,18 +2259,18 @@ TEST_F(ExprTest, CastExprs) {
   TestValue("cast(cast('2000-01-01 09:10:11.000000' as timestamp) as int)", TYPE_INT,
       946717811);
   TestTimestampValue("cast(946717811 as timestamp)",
-      TimestampValue("2000-01-01 09:10:11", 19));
+      TimestampValue::Parse("2000-01-01 09:10:11", 19));
 
   // Timestamp <--> Int conversions boundary cases
   TestValue("cast(cast('1400-01-01 00:00:00' as timestamp) as bigint)",
       TYPE_BIGINT, -17987443200);
   TestTimestampValue("cast(-17987443200 as timestamp)",
-      TimestampValue("1400-01-01 00:00:00", 19));
+      TimestampValue::Parse("1400-01-01 00:00:00", 19));
   TestIsNull("cast(-17987443201 as timestamp)", TYPE_TIMESTAMP);
   TestValue("cast(cast('9999-12-31 23:59:59' as timestamp) as bigint)",
       TYPE_BIGINT, 253402300799);
   TestTimestampValue("cast(253402300799 as timestamp)",
-      TimestampValue("9999-12-31 23:59:59", 19));
+      TimestampValue::Parse("9999-12-31 23:59:59", 19));
   TestIsNull("cast(253402300800 as timestamp)", TYPE_TIMESTAMP);
 
   // Timestamp <--> Float
@@ -2251,12 +2280,12 @@ TEST_F(ExprTest, CastExprs) {
   TestValue("cast(cast('2000-01-01 09:10:11.720000' as timestamp) as double)",
       TYPE_DOUBLE, 946717811.72);
   TestTimestampValue("cast(cast(946717811.033 as double) as timestamp)",
-      TimestampValue("2000-01-01 09:10:11.032999992", 29));
+      TimestampValue::Parse("2000-01-01 09:10:11.032999992", 29));
   TestValue("cast(cast('1400-01-01' as timestamp) as double)", TYPE_DOUBLE,
       -17987443200);
   TestIsNull("cast(cast(-17987443201.03 as double) as timestamp)", TYPE_TIMESTAMP);
   TestTimestampValue("cast(253402300799 as timestamp)",
-      TimestampValue("9999-12-31 23:59:59", 19));
+      TimestampValue::Parse("9999-12-31 23:59:59", 19));
   TestIsNull("cast(253433923200 as timestamp)", TYPE_TIMESTAMP);
   TestIsNull("cast(cast(null as bigint) as timestamp)", TYPE_TIMESTAMP);
   TestIsNull("cast(cast(null as timestamp) as bigint)", TYPE_BIGINT);
@@ -5092,7 +5121,7 @@ TEST_F(ExprTest, TimestampFunctions) {
       (posix_time::microsec_clock::local_time() - from_time_t(0)).total_seconds();
   timestamp_result = ConvertValue<TimestampValue>(GetValue(
       "cast(unix_timestamp() as timestamp)", TYPE_TIMESTAMP));
-  EXPECT_BETWEEN(TimestampValue(unix_start_time - 1), timestamp_result,
+  EXPECT_BETWEEN(TimestampValue::FromUnixTime(unix_start_time - 1), timestamp_result,
       TimestampValue::LocalTime());
 
   // Test alias
@@ -5592,8 +5621,8 @@ TEST_F(ExprTest, ConditionalFunctions) {
   TestValue("if(FALSE, cast(5.5 as double), cast(8.8 as double))", TYPE_DOUBLE, 8.8);
   TestStringValue("if(TRUE, 'abc', 'defgh')", "abc");
   TestStringValue("if(FALSE, 'abc', 'defgh')", "defgh");
-  TimestampValue then_val(1293872461);
-  TimestampValue else_val(929387245);
+  TimestampValue then_val = TimestampValue::FromUnixTime(1293872461);
+  TimestampValue else_val = TimestampValue::FromUnixTime(929387245);
   TestTimestampValue("if(TRUE, cast('2011-01-01 09:01:01' as timestamp), "
       "cast('1999-06-14 19:07:25' as timestamp))", then_val);
   TestTimestampValue("if(FALSE, cast('2011-01-01 09:01:01' as timestamp), "
@@ -5619,7 +5648,7 @@ TEST_F(ExprTest, ConditionalFunctions) {
   TestStringValue("nullif('abc', NULL)", "abc");
   TestIsNull("nullif(cast('2011-01-01 09:01:01' as timestamp), "
       "cast('2011-01-01 09:01:01' as timestamp))", TYPE_TIMESTAMP);
-  TimestampValue testlhs(1293872461);
+  TimestampValue testlhs = TimestampValue::FromUnixTime(1293872461);
   TestTimestampValue("nullif(cast('2011-01-01 09:01:01' as timestamp), "
       "cast('1999-06-14 19:07:25' as timestamp))", testlhs);
   TestIsNull("nullif(NULL, "
@@ -5682,8 +5711,8 @@ TEST_F(ExprTest, ConditionalFunctions) {
   TestStringValue("coalesce(NULL, 'abc', NULL)", "abc");
   TestStringValue("coalesce('defgh', NULL, 'abc', NULL)", "defgh");
   TestStringValue("coalesce(NULL, NULL, NULL, 'abc', NULL, NULL)", "abc");
-  TimestampValue ats(1293872461);
-  TimestampValue bts(929387245);
+  TimestampValue ats = TimestampValue::FromUnixTime(1293872461);
+  TimestampValue bts = TimestampValue::FromUnixTime(929387245);
   TestTimestampValue("coalesce(cast('2011-01-01 09:01:01' as timestamp))", ats);
   TestTimestampValue("coalesce(NULL, cast('2011-01-01 09:01:01' as timestamp),"
       "NULL)", ats);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/expr-value.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-value.h b/be/src/exprs/expr-value.h
index a2336d3..fa447cb 100644
--- a/be/src/exprs/expr-value.h
+++ b/be/src/exprs/expr-value.h
@@ -65,7 +65,6 @@ struct ExprValue {
   ExprValue(int64_t v) : bigint_val(v) {}
   ExprValue(float v) : float_val(v) {}
   ExprValue(double v) : double_val(v) {}
-  ExprValue(int64_t t, int64_t n) : timestamp_val(t, n) {}
 
   void Init(const std::string& str) {
     string_data = str;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/kudu-partition-expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/kudu-partition-expr.cc b/be/src/exprs/kudu-partition-expr.cc
index 2faaee5..ea295af 100644
--- a/be/src/exprs/kudu-partition-expr.cc
+++ b/be/src/exprs/kudu-partition-expr.cc
@@ -70,10 +70,10 @@ IntVal KuduPartitionExpr::GetIntVal(ExprContext* ctx, const TupleRow* row) {
     int col = tkudu_partition_expr_.referenced_columns[i];
     const ColumnDescriptor& col_desc = table_desc_->col_descs()[col];
     PrimitiveType type = col_desc.type().type;
-    Status s = WriteKuduRowValue(row_.get(), col, type, val, false);
+    Status s = WriteKuduValue(col, type, val, false, row_.get());
     // This can only fail if we set a col to an incorect type, which would be a bug in
     // planning, so we can DCHECK.
-    DCHECK(s.ok()) << "WriteKuduRowValue failed for col = " << col_desc.name()
+    DCHECK(s.ok()) << "WriteKuduValue failed for col = " << col_desc.name()
                    << " and type = " << col_desc.type() << ": " << s.GetDetail();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/literal.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/literal.cc b/be/src/exprs/literal.cc
index 4ae8a74..4fcf9fc 100644
--- a/be/src/exprs/literal.cc
+++ b/be/src/exprs/literal.cc
@@ -177,7 +177,7 @@ Literal::Literal(ColumnType type, double v)
   if (type.type == TYPE_DOUBLE) {
     value_.double_val = v;
   } else if (type.type == TYPE_TIMESTAMP) {
-    value_.timestamp_val = TimestampValue(v);
+    value_.timestamp_val = TimestampValue::FromSubsecondUnixTime(v);
   } else if (type.type == TYPE_DECIMAL) {
     bool overflow = false;
     switch (type.GetByteSize()) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/timestamp-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions-ir.cc b/be/src/exprs/timestamp-functions-ir.cc
index 3a6ea4f..6a80e71 100644
--- a/be/src/exprs/timestamp-functions-ir.cc
+++ b/be/src/exprs/timestamp-functions-ir.cc
@@ -25,6 +25,7 @@
 
 #include "exprs/anyval-util.h"
 #include "runtime/string-value.inline.h"
+#include "runtime/timestamp-value.inline.h"
 #include "runtime/timestamp-parse-util.h"
 #include "runtime/timestamp-value.h"
 #include "udf/udf.h"
@@ -77,8 +78,8 @@ StringVal TimestampFunctions::StringValFromTimestamp(FunctionContext* context,
 template <class TIME>
 StringVal TimestampFunctions::FromUnix(FunctionContext* context, const TIME& intp) {
   if (intp.is_null) return StringVal::null();
-  TimestampValue t(intp.val);
-  return AnyValUtil::FromString(context, lexical_cast<string>(t));
+  return AnyValUtil::FromString(context,
+      TimestampValue::FromUnixTime(intp.val).ToString());
 }
 
 template <class TIME>
@@ -90,7 +91,7 @@ StringVal TimestampFunctions::FromUnix(FunctionContext* context, const TIME& int
   }
   if (intp.is_null) return StringVal::null();
 
-  TimestampValue t(intp.val);
+  const TimestampValue& t = TimestampValue::FromUnixTime(intp.val);
   return StringValFromTimestamp(context, t, fmt);
 }
 
@@ -122,7 +123,7 @@ BigIntVal TimestampFunctions::Unix(FunctionContext* context) {
 TimestampVal TimestampFunctions::ToTimestamp(FunctionContext* context,
     const BigIntVal& bigint_val) {
   if (bigint_val.is_null) return TimestampVal::null();
-  TimestampValue tv(bigint_val.val);
+  const TimestampValue& tv = TimestampValue::FromUnixTime(bigint_val.val);
   TimestampVal tv_val;
   tv.ToTimestampVal(&tv_val);
   return tv_val;
@@ -144,7 +145,7 @@ TimestampVal TimestampFunctions::ToTimestamp(FunctionContext* context,
        return TimestampVal::null();
      }
   }
-  TimestampValue tv = TimestampValue(
+  const TimestampValue& tv = TimestampValue::Parse(
       reinterpret_cast<const char*>(date.ptr), date.len, *dt_ctx);
   TimestampVal tv_val;
   tv.ToTimestampVal(&tv_val);
@@ -154,7 +155,7 @@ TimestampVal TimestampFunctions::ToTimestamp(FunctionContext* context,
 StringVal TimestampFunctions::FromTimestamp(FunctionContext* context,
     const TimestampVal& date, const StringVal& fmt) {
   if (date.is_null) return StringVal::null();
-  TimestampValue tv = TimestampValue::FromTimestampVal(date);
+  const TimestampValue& tv = TimestampValue::FromTimestampVal(date);
   if (!tv.HasDate()) return StringVal::null();
   return StringValFromTimestamp(context, tv, fmt);
 }
@@ -162,7 +163,8 @@ StringVal TimestampFunctions::FromTimestamp(FunctionContext* context,
 BigIntVal TimestampFunctions::UnixFromString(FunctionContext* context,
     const StringVal& sv) {
   if (sv.is_null) return BigIntVal::null();
-  TimestampValue tv(reinterpret_cast<const char *>(sv.ptr), sv.len);
+  const TimestampValue& tv = TimestampValue::Parse(
+      reinterpret_cast<const char *>(sv.ptr), sv.len);
   time_t result;
   return (tv.ToUnixTime(&result)) ? BigIntVal(result) : BigIntVal::null();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/exprs/timestamp-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.cc b/be/src/exprs/timestamp-functions.cc
index b45610e..c5eb4b1 100644
--- a/be/src/exprs/timestamp-functions.cc
+++ b/be/src/exprs/timestamp-functions.cc
@@ -106,7 +106,7 @@ TimestampVal TimestampFunctions::FromUtc(FunctionContext* context,
   } catch (boost::exception&) {
     const string& msg = Substitute(
         "Timestamp '$0' did not convert to a valid local time in timezone '$1'",
-        ts_value.DebugString(), tz_string_value.DebugString());
+        ts_value.ToString(), tz_string_value.DebugString());
     context->AddWarning(msg.c_str());
     return TimestampVal::null();
   }
@@ -154,7 +154,7 @@ TimestampVal TimestampFunctions::ToUtc(FunctionContext* context,
   } catch (boost::exception&) {
     const string& msg =
         Substitute("Timestamp '$0' in timezone '$1' could not be converted to UTC",
-            ts_value.DebugString(), tz_string_value.DebugString());
+            ts_value.ToString(), tz_string_value.DebugString());
     context->AddWarning(msg.c_str());
     return TimestampVal::null();
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/raw-value-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/raw-value-test.cc b/be/src/runtime/raw-value-test.cc
index 7cea538..9c71822 100644
--- a/be/src/runtime/raw-value-test.cc
+++ b/be/src/runtime/raw-value-test.cc
@@ -171,7 +171,7 @@ TEST_F(RawValueTest, TemplatizedHash) {
     RawValue::GetHashValue(&string_value,ColumnType::CreateVarcharType(
     ColumnType::MAX_VARCHAR_LENGTH), seed));
 
-  TimestampValue timestamp_value(253433923200);
+  TimestampValue timestamp_value = TimestampValue::FromUnixTime(253433923200);
   EXPECT_EQ(RawValue::GetHashValue<impala::TimestampValue>(
     &timestamp_value, TYPE_TIMESTAMP, seed),RawValue::GetHashValue(
     &timestamp_value, TYPE_TIMESTAMP, seed));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index c5e2f59..f632b23 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -77,8 +77,7 @@ RuntimeState::RuntimeState(QueryState* query_state, const TPlanFragmentCtx& frag
   : query_state_(query_state),
     fragment_ctx_(&fragment_ctx),
     instance_ctx_(&instance_ctx),
-    now_(new TimestampValue(query_state->query_ctx().now_string.c_str(),
-        query_state->query_ctx().now_string.size())),
+    now_(new TimestampValue(TimestampValue::Parse(query_state->query_ctx().now_string))),
     exec_env_(exec_env),
     profile_(obj_pool(), "Fragment " + PrintId(instance_ctx.fragment_instance_id)),
     instance_buffer_reservation_(nullptr),
@@ -93,7 +92,7 @@ RuntimeState::RuntimeState(
     fragment_ctx_(nullptr),
     instance_ctx_(nullptr),
     local_query_state_(query_state_),
-    now_(new TimestampValue(qctx.now_string.c_str(), qctx.now_string.size())),
+    now_(new TimestampValue(TimestampValue::Parse(qctx.now_string))),
     exec_env_(exec_env),
     profile_(obj_pool(), "<unnamed>"),
     instance_buffer_reservation_(nullptr),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/timestamp-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-test.cc b/be/src/runtime/timestamp-test.cc
index 367fbd7..32fc60f 100644
--- a/be/src/runtime/timestamp-test.cc
+++ b/be/src/runtime/timestamp-test.cc
@@ -25,6 +25,7 @@
 #include "runtime/raw-value.inline.h"
 #include "runtime/timestamp-parse-util.h"
 #include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "testutil/gtest-util.h"
 #include "util/string-parser.h"
 
@@ -34,6 +35,7 @@ using boost::assign::list_of;
 using boost::date_time::Dec;
 using boost::date_time::not_a_date_time;
 using boost::gregorian::date;
+using boost::posix_time::ptime;
 using boost::posix_time::time_duration;
 
 namespace impala {
@@ -180,7 +182,7 @@ void TestTimestampTokens(vector<TimestampToken>* toks, int year, int month,
       string fmt_val = "Format: " + fmt + ", Val: " + val;
       DateTimeFormatContext dt_ctx(fmt.c_str(), fmt.length());
       ASSERT_TRUE(TimestampParser::ParseFormatTokens(&dt_ctx)) << fmt_val;
-      TimestampValue tv(val.c_str(), val.length(), dt_ctx);
+      TimestampValue tv = TimestampValue::Parse(val.c_str(), val.length(), dt_ctx);
       ValidateTimestamp(tv, fmt, val, fmt_val, year, month, day, hours, mins, secs,
           frac);
       int buff_len = dt_ctx.fmt_out_len + 1;
@@ -210,7 +212,7 @@ void TestTimestampTokens(vector<TimestampToken>* toks, int year, int month,
         string fmt_val = "Format: " + fmt + ", Val: " + val;
         DateTimeFormatContext dt_ctx(fmt.c_str(), fmt.length());
         ASSERT_TRUE(TimestampParser::ParseFormatTokens(&dt_ctx)) << fmt_val;
-        TimestampValue tv(val.c_str(), val.length(), dt_ctx);
+        TimestampValue tv = TimestampValue::Parse(val.c_str(), val.length(), dt_ctx);
         ValidateTimestamp(tv, fmt, val, fmt_val, year, month, day, hours, mins, secs,
             frac);
         int buff_len = dt_ctx.fmt_out_len + 1;
@@ -232,9 +234,9 @@ TEST(TimestampTest, Basic) {
   char s2[] = "1990-10-20 10:10:10.123456789  ";
   char s3[] = "  1990-10-20 10:10:10.123456789";
 
-  TimestampValue v1(s1, strlen(s1));
-  TimestampValue v2(s2, strlen(s2));
-  TimestampValue v3(s3, strlen(s3));
+  TimestampValue v1 = TimestampValue::Parse(s1, strlen(s1));
+  TimestampValue v2 = TimestampValue::Parse(s2, strlen(s2));
+  TimestampValue v3 = TimestampValue::Parse(s3, strlen(s3));
 
   EXPECT_EQ(v1.date().year(), 2012);
   EXPECT_EQ(v1.date().month(), 1);
@@ -260,8 +262,8 @@ TEST(TimestampTest, Basic) {
   char s4[] = "2012-01-20T01:10:01";
   char s5[] = "1990-10-20T10:10:10.123456789";
 
-  TimestampValue v4(s4, strlen(s4));
-  TimestampValue v5(s5, strlen(s5));
+  TimestampValue v4 = TimestampValue::Parse(s4, strlen(s4));
+  TimestampValue v5 = TimestampValue::Parse(s5, strlen(s5));
 
   EXPECT_EQ(v4.date().year(), 2012);
   EXPECT_EQ(v4.date().month(), 1);
@@ -281,8 +283,8 @@ TEST(TimestampTest, Basic) {
   // Test Dates and Times as timestamps.
   char d1[] = "2012-01-20";
   char d2[] = "1990-10-20";
-  TimestampValue dv1(d1, strlen(d1));
-  TimestampValue dv2(d2, strlen(d2));
+  TimestampValue dv1 = TimestampValue::Parse(d1, strlen(d1));
+  TimestampValue dv2 = TimestampValue::Parse(d2, strlen(d2));
 
   EXPECT_NE(dv1, dv2);
   EXPECT_LT(dv1, v1);
@@ -297,8 +299,8 @@ TEST(TimestampTest, Basic) {
 
   char t1[] = "10:11:12.123456789";
   char t2[] = "00:00:00";
-  TimestampValue tv1(t1, strlen(t1));
-  TimestampValue tv2(t2, strlen(t2));
+  TimestampValue tv1 = TimestampValue::Parse(t1, strlen(t1));
+  TimestampValue tv2 = TimestampValue::Parse(t2, strlen(t2));
 
   EXPECT_NE(tv1, tv2);
   EXPECT_NE(tv1, v2);
@@ -322,7 +324,7 @@ TEST(TimestampTest, Basic) {
       memcpy(frac_buff, TEST_VALS[i], VAL_LEN);
       memcpy(frac_buff + VAL_LEN, FRACTION_MAX_STR, fraction_len);
       *(frac_buff + VAL_LEN + fraction_len) = '\0';
-      TimestampValue tv_frac(frac_buff, strlen(frac_buff));
+      TimestampValue tv_frac = TimestampValue::Parse(frac_buff, strlen(frac_buff));
       if (frac_buff[4] == '-') {
         EXPECT_EQ(tv_frac.date().year(), 2013);
         EXPECT_EQ(tv_frac.date().month(), 12);
@@ -343,44 +345,44 @@ TEST(TimestampTest, Basic) {
 
   // Bad formats
   char b1[] = "1990-10 10:10:10.123456789";
-  TimestampValue bv1(b1, strlen(b1));
+  TimestampValue bv1 = TimestampValue::Parse(b1, strlen(b1));
   boost::gregorian::date not_a_date;
 
   EXPECT_EQ(bv1.date(), not_a_date);
   EXPECT_EQ(bv1.time(), not_a_date_time);
 
   char b2[] = "1991-10-10 99:10:10.123456789";
-  TimestampValue bv2(b2, strlen(b2));
+  TimestampValue bv2 = TimestampValue::Parse(b2, strlen(b2));
 
   EXPECT_EQ(bv2.time(), not_a_date_time);
   EXPECT_EQ(bv2.date(), not_a_date);
 
   char b3[] = "1990-10- 10:10:10.123456789";
-  TimestampValue bv3(b3, strlen(b3));
+  TimestampValue bv3 = TimestampValue::Parse(b3, strlen(b3));
 
   EXPECT_EQ(bv3.date(), not_a_date);
   EXPECT_EQ(bv3.time(), not_a_date_time);
 
   char b4[] = "10:1010.123456789";
-  TimestampValue bv4(b4, strlen(b4));
+  TimestampValue bv4 = TimestampValue::Parse(b4, strlen(b4));
 
   EXPECT_EQ(bv4.date(), not_a_date);
   EXPECT_EQ(bv4.time(), not_a_date_time);
 
   char b5[] = "10:11:12.123456 1991-10-10";
-  TimestampValue bv5(b5, strlen(b5));
+  TimestampValue bv5 = TimestampValue::Parse(b5, strlen(b5));
 
   EXPECT_EQ(bv5.date(), not_a_date);
   EXPECT_EQ(bv5.time(), not_a_date_time);
 
   char b6[] = "2012-01-20 01:10:00.123.466";
-  TimestampValue bv6(b6, strlen(b6));
+  TimestampValue bv6 = TimestampValue::Parse(b6, strlen(b6));
 
   EXPECT_EQ(bv6.date(), not_a_date);
   EXPECT_EQ(bv6.time(), not_a_date_time);
 
   char b7[] = "2012-01-20 01:10:00.123 477 ";
-  TimestampValue bv7(b7, strlen(b7));
+  TimestampValue bv7 = TimestampValue::Parse(b7, strlen(b7));
 
   EXPECT_EQ(bv7.date(), not_a_date);
   EXPECT_EQ(bv7.time(), not_a_date_time);
@@ -506,7 +508,7 @@ TEST(TimestampTest, Basic) {
     } else {
       ASSERT_TRUE(parse_result) << "TC: " << i;
     }
-    TimestampValue cust_tv(test_case.str, strlen(test_case.str), dt_ctx);
+    TimestampValue cust_tv = TimestampValue::Parse(test_case.str, strlen(test_case.str), dt_ctx);
     boost::gregorian::date cust_date = cust_tv.date();
     boost::posix_time::time_duration cust_time = cust_tv.time();
     if (test_case.str_should_fail) {
@@ -574,7 +576,7 @@ TEST(TimestampTest, Basic) {
     TimestampFormatTC test_case = fmt_test_cases[i];
     DateTimeFormatContext dt_ctx(test_case.fmt, strlen(test_case.fmt));
     ASSERT_TRUE(TimestampParser::ParseFormatTokens(&dt_ctx))  << "TC: " << i;
-    TimestampValue cust_tv(test_case.ts);
+    TimestampValue cust_tv = TimestampValue::FromUnixTime(test_case.ts);
     EXPECT_NE(cust_tv.date(), not_a_date) << "TC: " << i;
     EXPECT_NE(cust_tv.time(), not_a_date_time) << "TC: " << i;
     EXPECT_GE(dt_ctx.fmt_out_len, dt_ctx.fmt_len);
@@ -586,39 +588,119 @@ TEST(TimestampTest, Basic) {
     EXPECT_EQ(string(buff, actual_len),
         string(test_case.str, strlen(test_case.str))) << "TC: " << i;
   }
+
   // Test edge cases
-  TimestampValue min_date = TimestampValue("1400-01-01", 10);
+  const int64_t MIN_DATE_AS_UNIX_TIME = -17987443200;
+  TimestampValue min_date = TimestampValue::Parse("1400-01-01");
   EXPECT_TRUE(min_date.HasDate());
   EXPECT_TRUE(min_date.HasTime());
   time_t tm_min;
   EXPECT_TRUE(min_date.ToUnixTime(&tm_min));
-  EXPECT_EQ(-17987443200, tm_min);
-  EXPECT_EQ("1400-01-01 00:00:00", TimestampValue(-17987443200).DebugString());
-  TimestampValue too_early(-17987443201);
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME, tm_min);
+  int64_t tm_min_micros;
+  EXPECT_TRUE(min_date.UtcToUnixTimeMicros(&tm_min_micros));
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME * MICROS_PER_SEC, tm_min_micros);
+
+  // Add 250ns and check the value is rounded down
+  min_date = TimestampValue::FromUnixTimeNanos(MIN_DATE_AS_UNIX_TIME, 250);
+  EXPECT_TRUE(min_date.ToUnixTime(&tm_min));
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME, tm_min);
+  EXPECT_TRUE(min_date.UtcToUnixTimeMicros(&tm_min_micros));
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME * MICROS_PER_SEC, tm_min_micros);
+
+  // Add another 250ns and check the value is rounded up to the nearest microsecond.
+  EXPECT_TRUE(min_date.ToUnixTime(&tm_min));
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME, tm_min);
+  min_date.set_time(min_date.time() + boost::posix_time::nanoseconds(250));
+  EXPECT_TRUE(min_date.UtcToUnixTimeMicros(&tm_min_micros));
+  EXPECT_EQ(MIN_DATE_AS_UNIX_TIME * MICROS_PER_SEC + 1, tm_min_micros);
+
+  EXPECT_EQ("1400-01-01 00:00:00",
+      TimestampValue::FromUnixTime(MIN_DATE_AS_UNIX_TIME).ToString());
+  TimestampValue too_early = TimestampValue::FromUnixTime(MIN_DATE_AS_UNIX_TIME - 1);
   EXPECT_FALSE(too_early.HasDate());
   EXPECT_FALSE(too_early.HasTime());
-  // Boost's max supported year is 9999.
+  EXPECT_FALSE(too_early.UtcToUnixTimeMicros(&tm_min_micros));
+
+  // Test the max supported date that can be represented in seconds.
+  const int64_t MAX_DATE_AS_UNIX_TIME = 253402300799;
   TimestampValue max_date =
       TimestampValue(date(9999, Dec, 31), time_duration(23, 59, 59));
   EXPECT_TRUE(max_date.HasDate());
   EXPECT_TRUE(max_date.HasTime());
   time_t tm_max;
   EXPECT_TRUE(max_date.ToUnixTime(&tm_max));
-  EXPECT_EQ(253402300799, tm_max);
-  EXPECT_EQ("9999-12-31 23:59:59", TimestampValue(253402300799).DebugString());
-  TimestampValue too_late(253402300800);
+  EXPECT_EQ(MAX_DATE_AS_UNIX_TIME, tm_max);
+  int64_t tm_max_micros;
+  EXPECT_TRUE(max_date.UtcToUnixTimeMicros(&tm_max_micros));
+  EXPECT_EQ(MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC, tm_max_micros);
+
+  // Add 250 nanoseconds and test the result of UtcToUnixTimeMicros
+  max_date.set_time(max_date.time() + boost::posix_time::nanoseconds(250));
+  EXPECT_TRUE(max_date.UtcToUnixTimeMicros(&tm_max_micros));
+  EXPECT_EQ(MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC, tm_max_micros);
+  // Adding another 250ns will result in the timestamp being rounded up.
+  max_date.set_time(max_date.time() + boost::posix_time::nanoseconds(250));
+  EXPECT_TRUE(max_date.UtcToUnixTimeMicros(&tm_max_micros));
+  EXPECT_EQ(MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC + 1, tm_max_micros);
+
+  // The max date that can be represented with the maximum number of nanoseconds. Unlike
+  // the cases above, converting to microseconds does not round up to the next
+  // microsecond because that time is not supported by Impala.
+  max_date = TimestampValue::FromUnixTimeNanos(MAX_DATE_AS_UNIX_TIME, 999999999);
+  EXPECT_TRUE(max_date.HasDate());
+  EXPECT_TRUE(max_date.HasTime());
+  // The result is the maximum date with the maximum number of microseconds supported by
+  // Impala.
+  EXPECT_TRUE(max_date.UtcToUnixTimeMicros(&tm_max_micros));
+  EXPECT_EQ(MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC + 999999, tm_max_micros);
+
+  EXPECT_EQ("9999-12-31 23:59:59",
+      TimestampValue::FromUnixTime(MAX_DATE_AS_UNIX_TIME).ToString());
+  TimestampValue too_late = TimestampValue::FromUnixTime(MAX_DATE_AS_UNIX_TIME + 1);
   EXPECT_FALSE(too_late.HasDate());
   EXPECT_FALSE(too_late.HasTime());
 
   // Regression tests for IMPALA-1676, Unix times overflow int32 during year 2038
-  EXPECT_EQ("2038-01-19 03:14:08", TimestampValue(2147483648).DebugString());
-  EXPECT_EQ("2038-01-19 03:14:09", TimestampValue(2147483649).DebugString());
+  EXPECT_EQ("2038-01-19 03:14:08",
+      TimestampValue::FromUnixTime(2147483648).ToString());
+  EXPECT_EQ("2038-01-19 03:14:09",
+      TimestampValue::FromUnixTime(2147483649).ToString());
+
+  // Test a leap second in 1998 represented by the UTC time 1998-12-31 23:59:60.
+  // Unix time cannot represent the leap second, which repeats 915148800.
+  EXPECT_EQ("1998-12-31 23:59:59",
+      TimestampValue::FromUnixTime(915148799).ToString());
+  EXPECT_EQ("1999-01-01 00:00:00",
+      TimestampValue::FromUnixTime(915148800).ToString());
+  // The leap second doesn't parse in Impala.
+  TimestampValue leap_tv = TimestampValue::Parse("1998-12-31 23:59:60.00");
+  EXPECT_FALSE(leap_tv.HasDateAndTime());
+
+  // The leap second can be parsed by ptime, though it is just converted to the time
+  // that the Unix time would represent (i.e. the second after the new year). This shows
+  // both times constructed via ptime compare equally.
+  ptime leap_ptime1 = boost::posix_time::time_from_string("1998-12-31 23:59:60");
+  ptime leap_ptime2 = boost::posix_time::time_from_string("1999-01-01 00:00:00");
+  TimestampValue leap_tv1 = TimestampValue(leap_ptime1);
+  TimestampValue leap_tv2 = TimestampValue(leap_ptime2);
+  EXPECT_TRUE(leap_tv1.HasDateAndTime());
+  EXPECT_TRUE(leap_tv1 == TimestampValue(leap_ptime2));
+  time_t leap_time_t;
+  EXPECT_TRUE(leap_tv1.ToUnixTime(&leap_time_t));
+  EXPECT_EQ(915148800, leap_time_t);
+  EXPECT_EQ("1999-01-01 00:00:00", leap_tv1.ToString());
+  EXPECT_TRUE(leap_tv2.ToUnixTime(&leap_time_t));
+  EXPECT_EQ(915148800, leap_time_t);
+  EXPECT_EQ("1999-01-01 00:00:00", leap_tv2.ToString());
 
   // Test Unix time as a float
   double result;
-  EXPECT_TRUE(TimestampValue("2013-10-21 06:43:12.07", 22).ToSubsecondUnixTime(&result));
+  EXPECT_TRUE(
+      TimestampValue::Parse("2013-10-21 06:43:12.07").ToSubsecondUnixTime(&result));
   EXPECT_EQ(1382337792.07, result);
-  EXPECT_EQ("1970-01-01 00:00:00.008000000", TimestampValue(0.008).DebugString());
+  EXPECT_EQ("1970-01-01 00:00:00.008000000",
+      TimestampValue::FromSubsecondUnixTime(0.008).ToString());
 }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/timestamp-value.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.cc b/be/src/runtime/timestamp-value.cc
index 5d1c8b9..b3acc56 100644
--- a/be/src/runtime/timestamp-value.cc
+++ b/be/src/runtime/timestamp-value.cc
@@ -59,16 +59,25 @@ namespace impala {
 const char* TimestampValue::LLVM_CLASS_NAME = "class.impala::TimestampValue";
 const double TimestampValue::ONE_BILLIONTH = 0.000000001;
 
-TimestampValue::TimestampValue(const char* str, int len) {
-  TimestampParser::Parse(str, len, &date_, &time_);
+TimestampValue TimestampValue::Parse(const char* str, int len) {
+  TimestampValue tv;
+  TimestampParser::Parse(str, len, &tv.date_, &tv.time_);
+  return tv;
 }
 
-TimestampValue::TimestampValue(const char* str, int len,
+TimestampValue TimestampValue::Parse(const string& str) {
+  return Parse(str.c_str(), str.size());
+}
+
+TimestampValue TimestampValue::Parse(const char* str, int len,
     const DateTimeFormatContext& dt_ctx) {
-  TimestampParser::Parse(str, len, dt_ctx, &date_, &time_);
+  TimestampValue tv;
+  TimestampParser::Parse(str, len, dt_ctx, &tv.date_, &tv.time_);
+  return tv;
 }
 
-int TimestampValue::Format(const DateTimeFormatContext& dt_ctx, int len, char* buff) const {
+int TimestampValue::Format(const DateTimeFormatContext& dt_ctx, int len, char* buff)
+    const {
   return TimestampParser::Format(dt_ctx, date_, time_, len, buff);
 }
 
@@ -85,7 +94,7 @@ Status TimestampValue::UtcToLocal() {
         time_.minutes() * SECONDS_IN_MINUTE +
         time_.seconds();
     tm temp;
-    if (UNLIKELY(NULL == localtime_r(&utc, &temp))) {
+    if (UNLIKELY(localtime_r(&utc, &temp) == nullptr)) {
       *this = ptime(not_a_date_time);
       return Status("Failed to convert timestamp to local time.");
     }
@@ -109,7 +118,7 @@ Status TimestampValue::UtcToLocal() {
 bool TimestampValue::FromUtc(const std::string& timezone_str) {
   DCHECK(HasDateAndTime());
   time_zone_ptr timezone = TimezoneDatabase::FindTimezone(timezone_str, *this, true);
-  if (UNLIKELY(timezone == NULL)) {
+  if (UNLIKELY(timezone == nullptr)) {
     *this = ptime(not_a_date_time);
     return false;
   }
@@ -118,7 +127,7 @@ bool TimestampValue::FromUtc(const std::string& timezone_str) {
 
 bool TimestampValue::FromUtc(time_zone_ptr timezone) {
   DCHECK(HasDateAndTime());
-  DCHECK(timezone != NULL);
+  DCHECK(timezone != nullptr);
   ptime temp;
   ToPtime(&temp);
   local_date_time lt(temp, timezone);
@@ -127,20 +136,20 @@ bool TimestampValue::FromUtc(time_zone_ptr timezone) {
 }
 
 ostream& operator<<(ostream& os, const TimestampValue& timestamp_value) {
-  return os << timestamp_value.DebugString();
+  return os << timestamp_value.ToString();
 }
 
-ptime TimestampValue::UnixTimeToPtime(time_t unix_time) const {
+ptime TimestampValue::UnixTimeToPtime(time_t unix_time) {
   /// Unix times are represented internally in boost as 32 bit ints which limits the
   /// range of dates to 1901-2038 (https://svn.boost.org/trac/boost/ticket/3109), so
   /// libc functions will be used instead.
   tm temp_tm;
   if (FLAGS_use_local_tz_for_unix_timestamp_conversions) {
-    if (UNLIKELY(localtime_r(&unix_time, &temp_tm) == NULL)) {
+    if (UNLIKELY(localtime_r(&unix_time, &temp_tm) == nullptr)) {
       return ptime(not_a_date_time);
     }
   } else {
-    if (UNLIKELY(gmtime_r(&unix_time, &temp_tm) == NULL)) {
+    if (UNLIKELY(gmtime_r(&unix_time, &temp_tm) == nullptr)) {
       return ptime(not_a_date_time);
     }
   }
@@ -151,7 +160,7 @@ ptime TimestampValue::UnixTimeToPtime(time_t unix_time) const {
   }
 }
 
-string TimestampValue::DebugString() const {
+string TimestampValue::ToString() const {
   stringstream ss;
   if (HasDate()) {
     ss << boost::gregorian::to_iso_extended_string(date_);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/timestamp-value.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.h b/be/src/runtime/timestamp-value.h
index bdd92d5..e7d4cfd 100644
--- a/be/src/runtime/timestamp-value.h
+++ b/be/src/runtime/timestamp-value.h
@@ -22,8 +22,6 @@
 #include <boost/date_time/compiler_config.hpp>
 #include <boost/date_time/gregorian/gregorian.hpp>
 #include <boost/date_time/local_time/local_time.hpp>
-#include <boost/date_time/posix_time/conversion.hpp>
-#include <boost/date_time/posix_time/posix_time_types.hpp>
 #include <ctime>
 #include <gflags/gflags.h>
 #include <string>
@@ -81,23 +79,46 @@ class TimestampValue {
       : time_(t.time_of_day()),
         date_(t.date()) {}
   TimestampValue(const TimestampValue& tv) : time_(tv.time_), date_(tv.date_) {}
-  TimestampValue(const char* str, int len);
-  TimestampValue(const char* str, int len, const DateTimeFormatContext& dt_ctx);
+
+  /// Constructors that parse from a date/time string. See TimestampParser for details
+  /// about the date-time format.
+  static TimestampValue Parse(const std::string& str);
+  static TimestampValue Parse(const char* str, int len);
+  static TimestampValue Parse(const char* str, int len,
+      const DateTimeFormatContext& dt_ctx);
 
   /// Unix time (seconds since 1970-01-01 UTC by definition) constructors.
-  /// Conversion to local time will be done if
-  /// FLAGS_use_local_tz_for_unix_timestamp_conversions is true.
-  TimestampValue(int64_t unix_time, int64_t nanos) {
+  /// Return the corresponding timestamp in the local timezone if
+  /// FLAGS_use_local_tz_for_unix_timestamp_conversions is true. Otherwise, return the
+  /// corresponding timestamp in UTC.
+  static TimestampValue FromUnixTime(time_t unix_time) {
+    return TimestampValue(UnixTimeToPtime(unix_time));
+  }
+
+  /// Same as FromUnixTime() above, but adds the specified number of nanoseconds to the
+  /// resulting TimestampValue.
+  static TimestampValue FromUnixTimeNanos(time_t unix_time, int64_t nanos) {
     boost::posix_time::ptime temp = UnixTimeToPtime(unix_time);
     temp += boost::posix_time::nanoseconds(nanos);
-    *this = temp;
+    return TimestampValue(temp);
   }
 
-  explicit TimestampValue(double unix_time) {
+  /// Same as FromUnixTime() above, but adds the specified number of microseconds to the
+  /// resulting TimestampValue.
+  static TimestampValue FromUnixTimeMicros(time_t unix_time, int64_t micros) {
+    boost::posix_time::ptime temp = UnixTimeToPtime(unix_time);
+    temp += boost::posix_time::microseconds(micros);
+    return TimestampValue(temp);
+  }
+
+  /// Returns a TimestampValue where the integer part of the specified 'unix_time'
+  /// specifies the number of seconds (see above), and the fractional part is converted
+  /// to nanoseconds and added to the resulting TimestampValue.
+  static TimestampValue FromSubsecondUnixTime(double unix_time) {
     const time_t unix_time_whole = unix_time;
     boost::posix_time::ptime temp = UnixTimeToPtime(unix_time_whole);
     temp += boost::posix_time::nanoseconds((unix_time - unix_time_whole) / ONE_BILLIONTH);
-    *this = temp;
+    return TimestampValue(temp);
   }
 
   /// Returns the current local time with microsecond accuracy. This should not be used
@@ -139,7 +160,7 @@ class TimestampValue {
   bool HasDateOrTime() const { return HasDate() || HasTime(); }
   bool HasDateAndTime() const { return HasDate() && HasTime(); }
 
-  std::string DebugString() const;
+  std::string ToString() const;
 
   /// Verifies that the timestamp date falls into a valid range (years 1400..9999).
   inline bool IsValidDate() const {
@@ -163,37 +184,29 @@ class TimestampValue {
   /// Returns the number of characters copied in to the buffer (minus the terminator)
   int Format(const DateTimeFormatContext& dt_ctx, int len, char* buff) const;
 
+  /// Interpret 'this' as a timestamp in UTC and convert to unix time.
+  /// Returns false if the conversion failed ('unix_time' will be undefined), otherwise
+  /// true.
+  bool UtcToUnixTime(time_t* unix_time) const;
+
+  /// Interpret 'this' as a timestamp in UTC and convert to unix time in microseconds.
+  /// Nanoseconds are rounded to the nearest microsecond supported by Impala. Returns
+  /// false if the conversion failed ('unix_time_micros' will be undefined), otherwise
+  /// true.
+  bool UtcToUnixTimeMicros(int64_t* unix_time_micros) const;
+
   /// Converts to Unix time (seconds since the Unix epoch) representation. The time
   /// zone interpretation of the TimestampValue instance is determined by
   /// FLAGS_use_local_tz_for_unix_timestamp_conversions. If the flag is true, the
   /// instance is interpreted as a local value. If the flag is false, UTC is assumed.
   /// Returns false if the conversion failed (unix_time will be undefined), otherwise
   /// true.
-  bool ToUnixTime(time_t* unix_time) const {
-    DCHECK(unix_time != NULL);
-    if (UNLIKELY(!HasDateAndTime())) return false;
-    const boost::posix_time::ptime temp(date_, time_);
-    tm temp_tm = boost::posix_time::to_tm(temp);
-    if (FLAGS_use_local_tz_for_unix_timestamp_conversions) {
-      *unix_time = mktime(&temp_tm);
-    } else {
-      *unix_time = timegm(&temp_tm);
-    }
-    return true;
-  }
+  bool ToUnixTime(time_t* unix_time) const;
 
   /// Converts to Unix time with fractional seconds.
   /// Returns false if the conversion failed (unix_time will be undefined), otherwise
   /// true.
-  bool ToSubsecondUnixTime(double* unix_time) const {
-    DCHECK(unix_time != NULL);
-    time_t temp;
-    if (UNLIKELY(!ToUnixTime(&temp))) return false;
-    *unix_time = static_cast<double>(temp);
-    DCHECK(HasTime());
-    *unix_time += time_.fractional_seconds() * ONE_BILLIONTH;
-    return true;
-  }
+  bool ToSubsecondUnixTime(double* unix_time) const;
 
   /// Converts from UTC to local time in-place. The caller must ensure the TimestampValue
   /// this function is called upon has both a valid date and time. Returns Status::OK() if
@@ -272,7 +285,7 @@ class TimestampValue {
   /// The time zone of the resulting ptime is determined by
   /// FLAGS_use_local_tz_for_unix_timestamp_conversions. If the flag is true, the value
   /// will be in the local time zone. If the flag is false, the value will be in UTC.
-  boost::posix_time::ptime UnixTimeToPtime(time_t unix_time) const;
+  static boost::posix_time::ptime UnixTimeToPtime(time_t unix_time);
 };
 
 /// This function must be called 'hash_value' to be picked up by boost.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/runtime/timestamp-value.inline.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.inline.h b/be/src/runtime/timestamp-value.inline.h
new file mode 100644
index 0000000..7cf4acd
--- /dev/null
+++ b/be/src/runtime/timestamp-value.inline.h
@@ -0,0 +1,101 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+
+#ifndef IMPALA_RUNTIME_TIMESTAMP_VALUE_INLINE_H
+#define IMPALA_RUNTIME_TIMESTAMP_VALUE_INLINE_H
+
+#include "runtime/timestamp-value.h"
+
+#include <boost/date_time/compiler_config.hpp>
+#include <boost/date_time/posix_time/conversion.hpp>
+
+#include "gutil/walltime.h"
+
+namespace impala {
+
+/// Interpret 'this' as a timestamp in UTC and convert to unix time.
+/// Returns false if the conversion failed ('unix_time' will be undefined), otherwise
+/// true.
+inline bool TimestampValue::UtcToUnixTime(time_t* unix_time) const {
+  DCHECK(unix_time != nullptr);
+  if (UNLIKELY(!HasDateAndTime())) return false;
+  const boost::posix_time::ptime temp(date_, time_);
+  tm temp_tm = boost::posix_time::to_tm(temp);
+  *unix_time = timegm(&temp_tm);
+  return true;
+}
+
+/// Interpret 'this' as a timestamp in UTC and convert to unix time in microseconds.
+/// Nanoseconds are rounded to the nearest microsecond supported by Impala. Returns
+/// false if the conversion failed ('unix_time_micros' will be undefined), otherwise
+/// true.
+inline bool TimestampValue::UtcToUnixTimeMicros(int64_t* unix_time_micros) const {
+  const static int64_t MAX_UNIXTIME = 253402300799; // 9999-12-31 23:59:59
+  const static int64_t MAX_UNIXTIME_MICROS =
+    MAX_UNIXTIME * MICROS_PER_SEC + (MICROS_PER_SEC - 1);
+
+  DCHECK(unix_time_micros != nullptr);
+  time_t unixtime_seconds;
+  if (UNLIKELY(!UtcToUnixTime(&unixtime_seconds))) return false;
+
+  *unix_time_micros =
+    (static_cast<int64_t>(unixtime_seconds) * MICROS_PER_SEC) +
+    ((time_.fractional_seconds() + (NANOS_PER_MICRO / 2)) / NANOS_PER_MICRO);
+
+  // Rounding may result in the timestamp being MAX_UNIXTIME_MICROS+1 and should be
+  // truncated.
+  DCHECK_LE(*unix_time_micros, MAX_UNIXTIME_MICROS + 1);
+  *unix_time_micros = std::min(MAX_UNIXTIME_MICROS, *unix_time_micros);
+  return true;
+}
+
+/// Converts to Unix time (seconds since the Unix epoch) representation. The time
+/// zone interpretation of the TimestampValue instance is determined by
+/// FLAGS_use_local_tz_for_unix_timestamp_conversions. If the flag is true, the
+/// instance is interpreted as a local value. If the flag is false, UTC is assumed.
+/// Returns false if the conversion failed (unix_time will be undefined), otherwise
+/// true.
+inline bool TimestampValue::ToUnixTime(time_t* unix_time) const {
+  DCHECK(unix_time != nullptr);
+  if (UNLIKELY(!HasDateAndTime())) return false;
+  const boost::posix_time::ptime temp(date_, time_);
+  tm temp_tm = boost::posix_time::to_tm(temp);
+  if (FLAGS_use_local_tz_for_unix_timestamp_conversions) {
+    *unix_time = mktime(&temp_tm);
+  } else {
+    *unix_time = timegm(&temp_tm);
+  }
+  return true;
+}
+
+/// Converts to Unix time with fractional seconds.
+/// Returns false if the conversion failed (unix_time will be undefined), otherwise
+/// true.
+inline bool TimestampValue::ToSubsecondUnixTime(double* unix_time) const {
+  DCHECK(unix_time != nullptr);
+  time_t temp;
+  if (UNLIKELY(!ToUnixTime(&temp))) return false;
+  *unix_time = static_cast<double>(temp);
+  DCHECK(HasTime());
+  *unix_time += time_.fractional_seconds() * ONE_BILLIONTH;
+  return true;
+}
+
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/service/client-request-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 2f447c7..26bb3f4 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -107,7 +107,7 @@ ClientRequestState::ClientRequestState(
     summary_profile_.AddInfoString("HiveServer2 Protocol Version",
         Substitute("V$0", 1 + session->hs2_version));
   }
-  summary_profile_.AddInfoString("Start Time", start_time().DebugString());
+  summary_profile_.AddInfoString("Start Time", start_time().ToString());
   summary_profile_.AddInfoString("End Time", "");
   summary_profile_.AddInfoString("Query Type", "N/A");
   summary_profile_.AddInfoString("Query State", PrintQueryState(query_state_));
@@ -570,7 +570,7 @@ void ClientRequestState::Done() {
 
   unique_lock<mutex> l(lock_);
   end_time_ = TimestampValue::LocalTime();
-  summary_profile_.AddInfoString("End Time", end_time().DebugString());
+  summary_profile_.AddInfoString("End Time", end_time().ToString());
   summary_profile_.AddInfoString("Query State", PrintQueryState(query_state_));
   query_events_->MarkEvent("Unregister query");
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/service/impala-http-handler.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 6ccb6df..6ba8429 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -27,6 +27,8 @@
 #include "runtime/exec-env.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/query-state.h"
+#include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "service/impala-server.h"
 #include "service/client-request-state.h"
 #include "thrift/protocol/TDebugProtocol.h"
@@ -289,10 +291,10 @@ void ImpalaHttpHandler::QueryStateToJson(const ImpalaServer::QueryStateRecord& r
       document->GetAllocator());
   value->AddMember("stmt_type", stmt_type, document->GetAllocator());
 
-  Value start_time(record.start_time.DebugString().c_str(), document->GetAllocator());
+  Value start_time(record.start_time.ToString().c_str(), document->GetAllocator());
   value->AddMember("start_time", start_time, document->GetAllocator());
 
-  Value end_time(record.end_time.DebugString().c_str(), document->GetAllocator());
+  Value end_time(record.end_time.ToString().c_str(), document->GetAllocator());
   value->AddMember("end_time", end_time, document->GetAllocator());
 
   const TimestampValue& end_timestamp =
@@ -460,17 +462,19 @@ void ImpalaHttpHandler::SessionsHandler(const Webserver::ArgumentMap& args,
     Value default_db(state->database.c_str(), document->GetAllocator());
     session_json.AddMember("default_database", default_db, document->GetAllocator());
 
-    TimestampValue local_start_time(session.second->start_time_ms / 1000);
+    TimestampValue local_start_time = TimestampValue::FromUnixTime(
+        session.second->start_time_ms / 1000);
     local_start_time.UtcToLocal();
-    Value start_time(local_start_time.DebugString().c_str(), document->GetAllocator());
+    Value start_time(local_start_time.ToString().c_str(), document->GetAllocator());
     session_json.AddMember("start_time", start_time, document->GetAllocator());
     session_json.AddMember(
         "start_time_sort", session.second->start_time_ms, document->GetAllocator());
 
-    TimestampValue local_last_accessed(session.second->last_accessed_ms / 1000);
+    TimestampValue local_last_accessed = TimestampValue::FromUnixTime(
+        session.second->last_accessed_ms / 1000);
     local_last_accessed.UtcToLocal();
     Value last_accessed(
-        local_last_accessed.DebugString().c_str(), document->GetAllocator());
+        local_last_accessed.ToString().c_str(), document->GetAllocator());
     session_json.AddMember("last_accessed", last_accessed, document->GetAllocator());
     session_json.AddMember(
         "last_accessed_sort", session.second->last_accessed_ms, document->GetAllocator());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 1b2e8fe..10099b1 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -53,6 +53,7 @@
 #include "runtime/exec-env.h"
 #include "runtime/lib-cache.h"
 #include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "runtime/tmp-file-mgr.h"
 #include "scheduling/scheduler.h"
 #include "service/impala-http-handler.h"
@@ -335,7 +336,7 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
   ImpaladMetrics::CreateMetrics(
       exec_env->metrics()->GetOrCreateChildGroup("impala-server"));
   ImpaladMetrics::IMPALA_SERVER_START_TIME->set_value(
-      TimestampValue::LocalTime().DebugString());
+      TimestampValue::LocalTime().ToString());
 
   ABORT_IF_ERROR(ExternalDataSourceExecutor::InitJNI(exec_env->metrics()));
 
@@ -452,7 +453,7 @@ Status ImpalaServer::LogAuditRecord(const ClientRequestState& request_state,
   writer.String("session_id");
   writer.String(PrintId(request_state.session_id()).c_str());
   writer.String("start_time");
-  writer.String(request_state.start_time().DebugString().c_str());
+  writer.String(request_state.start_time().ToString().c_str());
   writer.String("authorization_failure");
   writer.Bool(Frontend::IsAuthorizationError(request_state.query_status()));
   writer.String("status");
@@ -858,7 +859,7 @@ Status ImpalaServer::ExecuteInternal(
 
 void ImpalaServer::PrepareQueryContext(TQueryCtx* query_ctx) {
   query_ctx->__set_pid(getpid());
-  query_ctx->__set_now_string(TimestampValue::LocalTime().DebugString());
+  query_ctx->__set_now_string(TimestampValue::LocalTime().ToString());
   query_ctx->__set_start_unix_millis(UnixMillis());
   query_ctx->__set_coord_address(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port));
 
@@ -1080,10 +1081,11 @@ Status ImpalaServer::GetSessionState(const TUniqueId& session_id,
     if (mark_active) {
       lock_guard<mutex> session_lock(i->second->lock);
       if (i->second->expired) {
+        int64_t last_time_s = i->second->last_accessed_ms / 1000;
         stringstream ss;
         ss << "Client session expired due to more than " << i->second->session_timeout
            << "s of inactivity (last activity was at: "
-           << TimestampValue(i->second->last_accessed_ms / 1000).DebugString() << ").";
+           << TimestampValue::FromUnixTime(last_time_s).ToString() << ").";
         return Status(ss.str());
       }
       if (i->second->closed) return Status("Session is closed");
@@ -1788,7 +1790,7 @@ void ImpalaServer::RegisterSessionTimeout(int32_t session_timeout) {
         if (now - last_accessed_ms <= session_timeout_ms) continue;
         LOG(INFO) << "Expiring session: " << session_state.first << ", user:"
                   << session_state.second->connected_user << ", last active: "
-                  << TimestampValue(last_accessed_ms / 1000).DebugString();
+                  << TimestampValue::FromUnixTime(last_accessed_ms / 1000).ToString();
         session_state.second->expired = true;
         ImpaladMetrics::NUM_SESSIONS_EXPIRED->Increment(1L);
         // Since expired is true, no more queries will be added to the inflight list.
@@ -1864,10 +1866,11 @@ void ImpalaServer::RegisterSessionTimeout(int32_t session_timeout) {
           }
         } else if (!query_state->is_active()) {
           // Otherwise time to expire this query
+          int64_t last_active_s = query_state->last_active_ms() / 1000;
           VLOG_QUERY
               << "Expiring query due to client inactivity: " << expiration_event->second
               << ", last activity was at: "
-              << TimestampValue(query_state->last_active_ms() / 1000).DebugString();
+              << TimestampValue::FromUnixTime(last_active_s).ToString();
           const string& err_msg = Substitute(
               "Query $0 expired due to client inactivity (timeout is $1)",
               PrintId(expiration_event->second),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/statestore/statestore-subscriber.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore-subscriber.cc b/be/src/statestore/statestore-subscriber.cc
index cb46e3b..08c11ac 100644
--- a/be/src/statestore/statestore-subscriber.cc
+++ b/be/src/statestore/statestore-subscriber.cc
@@ -262,7 +262,7 @@ void StatestoreSubscriber::RecoveryModeChecker() {
       // we would otherwise have to cache updates here.
       last_recovery_duration_metric_->set_value(
           recovery_timer.ElapsedTime() / (1000.0 * 1000.0 * 1000.0));
-      last_recovery_time_metric_->set_value(TimestampValue::LocalTime().DebugString());
+      last_recovery_time_metric_->set_value(TimestampValue::LocalTime().ToString());
     }
 
     SleepForMs(SLEEP_INTERVAL_MS);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/util/bloom-filter.h
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index 153848c..5ebd9b5 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -29,6 +29,7 @@
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gutil/macros.h"
 #include "runtime/buffered-block-mgr.h"
+#include "util/hash-util.h"
 
 namespace impala {
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/util/dict-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/dict-test.cc b/be/src/util/dict-test.cc
index 686bbce..620f431 100644
--- a/be/src/util/dict-test.cc
+++ b/be/src/util/dict-test.cc
@@ -100,9 +100,9 @@ TEST(DictTest, TestStrings) {
 }
 
 TEST(DictTest, TestTimestamps) {
-  TimestampValue tv1("2011-01-01 09:01:01", 19);
-  TimestampValue tv2("2012-01-01 09:01:01", 19);
-  TimestampValue tv3("2011-01-01 09:01:02", 19);
+  TimestampValue tv1 = TimestampValue::Parse("2011-01-01 09:01:01");
+  TimestampValue tv2 = TimestampValue::Parse("2012-01-01 09:01:01");
+  TimestampValue tv3 = TimestampValue::Parse("2011-01-01 09:01:02");
 
   vector<TimestampValue> dict_values;
   dict_values.push_back(tv1);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a16a0fa8/be/src/util/promise.h
----------------------------------------------------------------------
diff --git a/be/src/util/promise.h b/be/src/util/promise.h
index 8098797..c6b8f15 100644
--- a/be/src/util/promise.h
+++ b/be/src/util/promise.h
@@ -21,10 +21,9 @@
 #include <algorithm>
 #include <boost/thread.hpp>
 
-#include "common/logging.h"
-#include "runtime/timestamp-value.h"
 #include "util/time.h"
 #include "common/atomic.h"
+#include "common/logging.h"
 
 namespace impala {