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 2017/08/17 03:14:21 UTC

[05/15] incubator-impala git commit: IMPALA-3931: arbitrary fixed-size uda intermediate types

IMPALA-3931: arbitrary fixed-size uda intermediate types

Make many builtin aggregate functions use fixed-length intermediate
types:
* avg()
* ndv()
* stddev(), variance(), etc
* distinctpc(), distinctpcsa()

sample(), appx_median(), histogram() and group_concat() actually
allocate var-len data so aren't changed.

This has some major benefits:
* Spill-to-disk works properly with these aggregations.
* Aggregations are more efficient because there is one less pointer
  indirection.
* Aggregations use less memory, because we don't need an extra 12-byte
  StringValue for the indirection.

Adds a special-purpose internal type FIXED_UDA_INTERMEDIATE. The type
is represented in the same way as CHAR - a fixed-size array of bytes,
stored inline in tuples. However, it is not user-visible and does
not support CHAR semantics, i.e. users can't declare tables, functions,
etc with the type. The pointer and length is passed into aggregate functions
wrapped in a StringVal.

Updates some internal codegen functions to work better with the new
type. E.g. store values directly into the result tuple instead of
via an intermediate stack allocation.

Testing:
This change only affects builtin aggregate functions, for which we
have test coverage already. If we were to allow wider use of this type,
it would need further testing.

Added an analyzer test to ensure we can't use the type for UDAs.

Added a regression test for spilling avg().

Added a regression test for UDA with CHAR intermediate hitting DCHECK.

Perf:
Ran TPC-H locally. TPC-H Q17, which has a high-cardinality AVG(),
improved dramatically.

+----------+-----------------------+---------+------------+------------+----------------+
| Workload | File Format           | Avg (s) | Delta(Avg) | GeoMean(s) | Delta(GeoMean) |
+----------+-----------------------+---------+------------+------------+----------------+
| TPCH(60) | parquet / none / none | 18.44   | -17.54%    | 11.92      | -5.34%         |
+----------+-----------------------+---------+------------+------------+----------------+

+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+
| Workload | Query    | File Format           | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%) | Base StdDev(%) | Num Clients | Iters |
+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+
| TPCH(60) | TPCH-Q12 | parquet / none / none | 18.40  | 17.64       | +4.32%     |   0.77%   |   1.09%        | 1           | 5     |
| TPCH(60) | TPCH-Q22 | parquet / none / none | 7.07   | 6.90        | +2.36%     |   0.28%   |   0.30%        | 1           | 5     |
| TPCH(60) | TPCH-Q3  | parquet / none / none | 12.37  | 12.11       | +2.10%     |   0.18%   |   0.15%        | 1           | 5     |
| TPCH(60) | TPCH-Q7  | parquet / none / none | 42.48  | 42.09       | +0.93%     |   2.45%   |   0.80%        | 1           | 5     |
| TPCH(60) | TPCH-Q6  | parquet / none / none | 3.18   | 3.15        | +0.89%     |   0.67%   |   0.76%        | 1           | 5     |
| TPCH(60) | TPCH-Q19 | parquet / none / none | 7.24   | 7.20        | +0.50%     |   0.95%   |   0.67%        | 1           | 5     |
| TPCH(60) | TPCH-Q10 | parquet / none / none | 13.37  | 13.30       | +0.50%     |   0.48%   |   1.39%        | 1           | 5     |
| TPCH(60) | TPCH-Q5  | parquet / none / none | 7.47   | 7.44        | +0.36%     |   0.58%   |   0.54%        | 1           | 5     |
| TPCH(60) | TPCH-Q11 | parquet / none / none | 2.03   | 2.02        | +0.06%     |   0.26%   |   1.95%        | 1           | 5     |
| TPCH(60) | TPCH-Q4  | parquet / none / none | 5.48   | 5.50        | -0.27%     |   0.62%   |   1.12%        | 1           | 5     |
| TPCH(60) | TPCH-Q13 | parquet / none / none | 22.11  | 22.18       | -0.31%     |   0.18%   |   0.55%        | 1           | 5     |
| TPCH(60) | TPCH-Q15 | parquet / none / none | 8.45   | 8.48        | -0.32%     |   0.40%   |   0.47%        | 1           | 5     |
| TPCH(60) | TPCH-Q9  | parquet / none / none | 33.39  | 33.66       | -0.81%     |   0.75%   |   0.59%        | 1           | 5     |
| TPCH(60) | TPCH-Q21 | parquet / none / none | 71.34  | 72.07       | -1.01%     |   1.84%   |   1.79%        | 1           | 5     |
| TPCH(60) | TPCH-Q14 | parquet / none / none | 5.93   | 6.00        | -1.07%     |   0.15%   |   0.69%        | 1           | 5     |
| TPCH(60) | TPCH-Q20 | parquet / none / none | 5.72   | 5.79        | -1.09%     |   0.59%   |   0.51%        | 1           | 5     |
| TPCH(60) | TPCH-Q18 | parquet / none / none | 45.42  | 45.93       | -1.10%     |   1.42%   |   0.50%        | 1           | 5     |
| TPCH(60) | TPCH-Q2  | parquet / none / none | 4.81   | 4.89        | -1.52%     |   1.68%   |   1.01%        | 1           | 5     |
| TPCH(60) | TPCH-Q16 | parquet / none / none | 5.41   | 5.52        | -1.98%     |   0.66%   |   0.73%        | 1           | 5     |
| TPCH(60) | TPCH-Q1  | parquet / none / none | 27.58  | 29.13       | -5.34%     |   0.24%   |   1.51%        | 1           | 5     |
| TPCH(60) | TPCH-Q8  | parquet / none / none | 12.61  | 14.30       | -11.78%    |   6.20%   | * 15.28% *     | 1           | 5     |
| TPCH(60) | TPCH-Q17 | parquet / none / none | 43.74  | 126.58      | I -65.44%  |   1.34%   |   9.60%        | 1           | 5     |
+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+

Change-Id: Ife90cf27989f98ffb5ef5c39f1e09ce92e8cb87c
Reviewed-on: http://gerrit.cloudera.org:8080/7526
Tested-by: Impala Public Jenkins
Reviewed-by: Tim Armstrong <ta...@cloudera.com>


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

Branch: refs/heads/master
Commit: 852e1bb728c790f27b49ebee2857f68860272c70
Parents: 79818f0
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Jul 25 18:16:32 2017 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Thu Aug 17 03:12:48 2017 +0000

----------------------------------------------------------------------
 be/src/codegen/codegen-anyval.cc                |  89 +++++---
 be/src/codegen/codegen-anyval.h                 |  30 +--
 be/src/codegen/llvm-codegen.cc                  |  13 +-
 be/src/codegen/llvm-codegen.h                   |   8 +-
 be/src/exec/hash-table.cc                       |   2 +-
 be/src/exec/incr-stats-util.cc                  |   1 -
 be/src/exec/partitioned-aggregation-node.cc     |  73 +++----
 be/src/exprs/agg-fn-evaluator.cc                |  28 ++-
 be/src/exprs/aggregate-functions-ir.cc          | 102 +++++-----
 be/src/exprs/anyval-util.cc                     |   6 +
 be/src/exprs/anyval-util.h                      |   5 +-
 be/src/exprs/scalar-expr-evaluator.cc           |   3 +-
 be/src/exprs/slot-ref.cc                        |  14 +-
 be/src/exprs/utility-functions-ir.cc            |   3 +-
 be/src/runtime/raw-value.cc                     |   2 +-
 be/src/runtime/types.cc                         |  19 +-
 be/src/runtime/types.h                          |  20 +-
 be/src/testutil/test-udas.cc                    |  36 ++++
 be/src/udf/udf.h                                |  20 +-
 common/thrift/Types.thrift                      |   7 +-
 .../org/apache/impala/catalog/BuiltinsDb.java   |  89 +++++---
 .../org/apache/impala/catalog/Function.java     |   2 +
 .../apache/impala/catalog/PrimitiveType.java    |   9 +-
 .../org/apache/impala/catalog/ScalarType.java   |  23 ++-
 .../java/org/apache/impala/catalog/Type.java    |  13 +-
 .../org/apache/impala/service/MetadataOp.java   |   3 +-
 .../apache/impala/analysis/AnalyzeDDLTest.java  |   3 +
 .../org/apache/impala/service/FrontendTest.java |   6 +-
 .../PlannerTest/spillable-buffer-sizing.test    |  12 +-
 .../queries/QueryTest/alloc-fail-init.test      |  35 ----
 .../queries/QueryTest/functions-ddl.test        |  12 +-
 .../queries/QueryTest/spilling-aggs.test        | 202 +++++++++++++++++++
 .../queries/QueryTest/spilling.test             | 180 -----------------
 .../functional-query/queries/QueryTest/uda.test |  24 ++-
 tests/query_test/test_spilling.py               |   3 +
 tests/query_test/test_udfs.py                   |   5 +
 36 files changed, 651 insertions(+), 451 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/codegen/codegen-anyval.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.cc b/be/src/codegen/codegen-anyval.cc
index a73faff..bdd49c9 100644
--- a/be/src/codegen/codegen-anyval.cc
+++ b/be/src/codegen/codegen-anyval.cc
@@ -52,6 +52,7 @@ Type* CodegenAnyVal::GetLoweredType(LlvmCodeGen* cg, const ColumnType& type) {
       return StructType::get(cg->tinyint_type(), cg->double_type(), NULL);
     case TYPE_STRING: // { i64, i8* }
     case TYPE_VARCHAR: // { i64, i8* }
+    case TYPE_FIXED_UDA_INTERMEDIATE: // { i64, i8* }
       return StructType::get(cg->bigint_type(), cg->ptr_type(), NULL);
     case TYPE_CHAR:
       DCHECK(false) << "NYI:" << type.DebugString();
@@ -97,6 +98,7 @@ Type* CodegenAnyVal::GetUnloweredType(LlvmCodeGen* cg, const ColumnType& type) {
       break;
     case TYPE_STRING:
     case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE:
       result = cg->GetType(LLVM_STRINGVAL_NAME);
       break;
     case TYPE_CHAR:
@@ -187,6 +189,7 @@ Value* CodegenAnyVal::GetIsNull(const char* name) const {
     }
     case TYPE_STRING:
     case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE:
     case TYPE_TIMESTAMP: {
       // Lowered type is of form { i64, *}. Get the first byte of the i64 value.
       Value* v = builder_->CreateExtractValue(value_, 0);
@@ -231,6 +234,7 @@ void CodegenAnyVal::SetIsNull(Value* is_null) {
     }
     case TYPE_STRING:
     case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE:
     case TYPE_TIMESTAMP: {
       // Lowered type is of the form { i64, * }. Set the first byte of the i64 value to
       // 'is_null'
@@ -267,6 +271,8 @@ Value* CodegenAnyVal::GetVal(const char* name) {
       << "Use GetPtr and GetLen for Varchar";
   DCHECK(type_.type != TYPE_CHAR)
       << "Use GetPtr and GetLen for Char";
+  DCHECK(type_.type != TYPE_FIXED_UDA_INTERMEDIATE)
+      << "Use GetPtr and GetLen for FixedUdaIntermediate";
   DCHECK(type_.type != TYPE_TIMESTAMP)
       << "Use GetDate and GetTimeOfDay for TimestampVals";
   switch(type_.type) {
@@ -310,6 +316,8 @@ void CodegenAnyVal::SetVal(Value* val) {
   DCHECK(type_.type != TYPE_STRING) << "Use SetPtr and SetLen for StringVals";
   DCHECK(type_.type != TYPE_VARCHAR) << "Use SetPtr and SetLen for StringVals";
   DCHECK(type_.type != TYPE_CHAR) << "Use SetPtr and SetLen for StringVals";
+  DCHECK(type_.type != TYPE_FIXED_UDA_INTERMEDIATE)
+      << "Use SetPtr and SetLen for FixedUdaIntermediate";
   DCHECK(type_.type != TYPE_TIMESTAMP)
       << "Use SetDate and SetTimeOfDay for TimestampVals";
   switch(type_.type) {
@@ -403,13 +411,13 @@ Value* CodegenAnyVal::GetLen() {
 
 void CodegenAnyVal::SetPtr(Value* ptr) {
   // Set the second pointer value to 'ptr'.
-  DCHECK(type_.IsStringType());
+  DCHECK(type_.IsStringType() || type_.type == TYPE_FIXED_UDA_INTERMEDIATE);
   value_ = builder_->CreateInsertValue(value_, ptr, 1, name_);
 }
 
 void CodegenAnyVal::SetLen(Value* len) {
   // Set the high bytes of the first value to 'len'.
-  DCHECK(type_.IsStringType());
+  DCHECK(type_.IsStringType() || type_.type == TYPE_FIXED_UDA_INTERMEDIATE);
   Value* v = builder_->CreateExtractValue(value_, 0);
   v = SetHighBits(32, len, v);
   value_ = builder_->CreateInsertValue(value_, v, 0, name_);
@@ -459,17 +467,26 @@ Value* CodegenAnyVal::GetUnloweredPtr(const string& name) const {
       GetLoweredPtr(), GetUnloweredPtrType(codegen_, type_), name);
 }
 
-void CodegenAnyVal::SetFromRawValue(Value* raw_val) {
-  DCHECK_EQ(raw_val->getType(), codegen_->GetType(type_))
+void CodegenAnyVal::LoadFromNativePtr(Value* raw_val_ptr) {
+  DCHECK(raw_val_ptr->getType()->isPointerTy());
+  Type* raw_val_type = raw_val_ptr->getType()->getPointerElementType();
+  DCHECK_EQ(raw_val_type, codegen_->GetType(type_))
       << endl
-      << LlvmCodeGen::Print(raw_val) << endl
+      << LlvmCodeGen::Print(raw_val_ptr) << endl
       << type_ << " => " << LlvmCodeGen::Print(codegen_->GetType(type_));
   switch (type_.type) {
     case TYPE_STRING:
     case TYPE_VARCHAR: {
       // Convert StringValue to StringVal
-      SetPtr(builder_->CreateExtractValue(raw_val, 0, "ptr"));
-      SetLen(builder_->CreateExtractValue(raw_val, 1, "len"));
+      Value* string_value = builder_->CreateLoad(raw_val_ptr, "string_value");
+      SetPtr(builder_->CreateExtractValue(string_value, 0, "ptr"));
+      SetLen(builder_->CreateExtractValue(string_value, 1, "len"));
+      break;
+    }
+    case TYPE_FIXED_UDA_INTERMEDIATE: {
+      // Convert fixed-size slot to StringVal.
+      SetPtr(builder_->CreateBitCast(raw_val_ptr, codegen_->ptr_type()));
+      SetLen(codegen_->GetIntConstant(TYPE_INT, type_.len));
       break;
     }
     case TYPE_CHAR:
@@ -481,15 +498,16 @@ void CodegenAnyVal::SetFromRawValue(Value* raw_val) {
       //   { boost::posix_time::time_duration, boost::gregorian::date }
       // = { {{{i64}}}, {{i32}} }
 
+      Value* ts_value = builder_->CreateLoad(raw_val_ptr, "ts_value");
       // Extract time_of_day i64 from boost::posix_time::time_duration.
       uint32_t time_of_day_idxs[] = {0, 0, 0, 0};
       Value* time_of_day =
-          builder_->CreateExtractValue(raw_val, time_of_day_idxs, "time_of_day");
+          builder_->CreateExtractValue(ts_value, time_of_day_idxs, "time_of_day");
       DCHECK(time_of_day->getType()->isIntegerTy(64));
       SetTimeOfDay(time_of_day);
       // Extract i32 from boost::gregorian::date
       uint32_t date_idxs[] = {1, 0, 0};
-      Value* date = builder_->CreateExtractValue(raw_val, date_idxs, "date");
+      Value* date = builder_->CreateExtractValue(ts_value, date_idxs, "date");
       DCHECK(date->getType()->isIntegerTy(32));
       SetDate(date);
       break;
@@ -502,8 +520,7 @@ void CodegenAnyVal::SetFromRawValue(Value* raw_val) {
     case TYPE_FLOAT:
     case TYPE_DOUBLE:
     case TYPE_DECIMAL:
-      // raw_val is a native type
-      SetVal(raw_val);
+      SetVal(builder_->CreateLoad(raw_val_ptr, "raw_val"));
       break;
     default:
       DCHECK(false) << "NYI: " << type_.DebugString();
@@ -511,36 +528,45 @@ void CodegenAnyVal::SetFromRawValue(Value* raw_val) {
   }
 }
 
-Value* CodegenAnyVal::ToNativeValue(Value* pool_val) {
+void CodegenAnyVal::StoreToNativePtr(Value* raw_val_ptr, Value* pool_val) {
   Type* raw_type = codegen_->GetType(type_);
-  Value* raw_val = Constant::getNullValue(raw_type);
   switch (type_.type) {
     case TYPE_STRING:
     case TYPE_VARCHAR: {
       // Convert StringVal to StringValue
+      Value* string_value = Constant::getNullValue(raw_type);
       Value* len = GetLen();
-      raw_val = builder_->CreateInsertValue(raw_val, len, 1);
+      string_value = builder_->CreateInsertValue(string_value, len, 1);
       if (pool_val == nullptr) {
-        // Set raw_val.ptr from this->ptr
-        raw_val = builder_->CreateInsertValue(raw_val, GetPtr(), 0);
+        // Set string_value.ptr from this->ptr
+        string_value = builder_->CreateInsertValue(string_value, GetPtr(), 0);
       } else {
-        // Allocate raw_val.ptr from 'pool_val' and copy this->ptr
+        // Allocate string_value.ptr from 'pool_val' and copy this->ptr
         Value* new_ptr =
             codegen_->CodegenMemPoolAllocate(builder_, pool_val, len, "new_ptr");
         codegen_->CodegenMemcpy(builder_, new_ptr, GetPtr(), len);
-        raw_val = builder_->CreateInsertValue(raw_val, new_ptr, 0);
+        string_value = builder_->CreateInsertValue(string_value, new_ptr, 0);
       }
+      builder_->CreateStore(string_value, raw_val_ptr);
       break;
     }
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      DCHECK(false) << "FIXED_UDA_INTERMEDIATE does not need to be copied: the "
+                    << "StringVal must be set up to point to the output slot";
+      break;
     case TYPE_TIMESTAMP: {
       // Convert TimestampVal to TimestampValue
       // TimestampValue has type
       //   { boost::posix_time::time_duration, boost::gregorian::date }
       // = { {{{i64}}}, {{i32}} }
+      Value* timestamp_value = Constant::getNullValue(raw_type);
       uint32_t time_of_day_idxs[] = {0, 0, 0, 0};
-      raw_val = builder_->CreateInsertValue(raw_val, GetTimeOfDay(), time_of_day_idxs);
+      timestamp_value =
+          builder_->CreateInsertValue(timestamp_value, GetTimeOfDay(), time_of_day_idxs);
       uint32_t date_idxs[] = {1, 0, 0};
-      raw_val = builder_->CreateInsertValue(raw_val, GetDate(), date_idxs);
+      timestamp_value =
+          builder_->CreateInsertValue(timestamp_value, GetDate(), date_idxs);
+      builder_->CreateStore(timestamp_value, raw_val_ptr);
       break;
     }
     case TYPE_BOOLEAN:
@@ -551,22 +577,19 @@ Value* CodegenAnyVal::ToNativeValue(Value* pool_val) {
     case TYPE_FLOAT:
     case TYPE_DOUBLE:
     case TYPE_DECIMAL:
-      // raw_val is a native type
-      raw_val = GetVal();
+      // The representations of the types match - just store the value.
+      builder_->CreateStore(GetVal(), raw_val_ptr);
       break;
     default:
       DCHECK(false) << "NYI: " << type_.DebugString();
       break;
   }
-  return raw_val;
 }
 
-Value* CodegenAnyVal::ToNativePtr(Value* native_ptr, Value* pool_val) {
-  Value* v = ToNativeValue(pool_val);
-  if (native_ptr == nullptr) {
-    native_ptr = codegen_->CreateEntryBlockAlloca(*builder_, v->getType());
-  }
-  builder_->CreateStore(v, native_ptr);
+Value* CodegenAnyVal::ToNativePtr(Value* pool_val) {
+  Value* native_ptr =
+      codegen_->CreateEntryBlockAlloca(*builder_, codegen_->GetType(type_));
+  StoreToNativePtr(native_ptr, pool_val);
   return native_ptr;
 }
 
@@ -611,7 +634,7 @@ void CodegenAnyVal::WriteToSlot(const SlotDescriptor& slot_desc, Value* tuple_va
   builder_->SetInsertPoint(non_null_block);
   Value* slot =
       builder_->CreateStructGEP(nullptr, tuple_val, slot_desc.llvm_field_idx(), "slot");
-  ToNativePtr(slot, pool_val);
+  StoreToNativePtr(slot, pool_val);
   builder_->CreateBr(insert_before);
 
   // Null block: set null bit
@@ -638,7 +661,8 @@ Value* CodegenAnyVal::Eq(CodegenAnyVal* other) {
     case TYPE_DOUBLE:
       return builder_->CreateFCmpUEQ(GetVal(), other->GetVal(), "eq");
     case TYPE_STRING:
-    case TYPE_VARCHAR: {
+    case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE: {
       Function* eq_fn =
           codegen_->GetFunction(IRFunction::CODEGEN_ANYVAL_STRING_VAL_EQ, false);
       return builder_->CreateCall(
@@ -675,7 +699,8 @@ Value* CodegenAnyVal::EqToNativePtr(Value* native_ptr) {
     case TYPE_DOUBLE:
       return builder_->CreateFCmpUEQ(GetVal(), val, "cmp_raw");
     case TYPE_STRING:
-    case TYPE_VARCHAR: {
+    case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE: {
       Function* eq_fn =
           codegen_->GetFunction(IRFunction::CODEGEN_ANYVAL_STRING_VALUE_EQ, false);
       return builder_->CreateCall(eq_fn,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/codegen/codegen-anyval.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.h b/be/src/codegen/codegen-anyval.h
index 063004a..2bc800e 100644
--- a/be/src/codegen/codegen-anyval.h
+++ b/be/src/codegen/codegen-anyval.h
@@ -49,7 +49,7 @@ namespace impala {
 /// TYPE_BIGINT/BigIntVal: { i8, i64 }
 /// TYPE_FLOAT/FloatVal: i64
 /// TYPE_DOUBLE/DoubleVal: { i8, double }
-/// TYPE_STRING/StringVal: { i64, i8* }
+/// TYPE_STRING,TYPE_VARCHAR,TYPE_CHAR,TYPE_FIXED_UDA_INTERMEDIATE/StringVal: { i64, i8* }
 /// TYPE_TIMESTAMP/TimestampVal: { i64, i64 }
 /// TYPE_DECIMAL/DecimalVal (isn't lowered):
 /// %"struct.impala_udf::DecimalVal" { {i8}, [15 x i8], {i128} }
@@ -190,26 +190,26 @@ class CodegenAnyVal {
   /// unlowered type. This *Val should be non-null. The output variable is called 'name'.
   llvm::Value* GetUnloweredPtr(const std::string& name = "") const;
 
-  /// Set this *Val's value based on 'raw_val'. 'raw_val' should be a native type,
-  /// StringValue, or TimestampValue.
-  void SetFromRawValue(llvm::Value* raw_val);
+  /// Load this *Val's value from 'raw_val_ptr', which must be a pointer to the matching
+  /// native type, e.g. a StringValue or TimestampValue slot in a tuple.
+  void LoadFromNativePtr(llvm::Value* raw_val_ptr);
 
-  /// Converts this *Val's value to a native type, StringValue, TimestampValue, etc.
+  /// Stores this *Val's value into a native slot, e.g. a StringValue or TimestampValue.
   /// This should only be used if this *Val is not null.
   ///
+  /// Not valid to call for FIXED_UDA_INTERMEDIATE: in that case the StringVal must be
+  /// set up to point directly to the underlying slot, e.g. by LoadFromNativePtr().
+  ///
   /// If 'pool_val' is non-NULL, var-len data will be copied into 'pool_val'.
   /// 'pool_val' has to be of type MemPool*.
-  llvm::Value* ToNativeValue(llvm::Value* pool_val = nullptr);
+  void StoreToNativePtr(llvm::Value* raw_val_ptr, llvm::Value* pool_val = nullptr);
 
-  /// Sets 'native_ptr' to this *Val's value. If non-NULL, 'native_ptr' should be a
-  /// pointer to a native type, StringValue, TimestampValue, etc. If NULL, a pointer is
-  /// alloca'd. In either case the pointer is returned. This should only be used if this
-  /// *Val is not null.
+  /// Creates a pointer, e.g. StringValue* to an alloca() allocation with the
+  /// equivalent of this value. This should only be used if this Val is not null.
   ///
   /// If 'pool_val' is non-NULL, var-len data will be copied into 'pool_val'.
   /// 'pool_val' has to be of type MemPool*.
-  llvm::Value* ToNativePtr(
-      llvm::Value* native_ptr = nullptr, llvm::Value* pool_val = nullptr);
+  llvm::Value* ToNativePtr(llvm::Value* pool_val = nullptr);
 
   /// Writes this *Val's value to the appropriate slot in 'tuple' if non-null, or sets the
   /// appropriate null bit if null. This assumes null bits are initialized to 0. Analogous
@@ -230,9 +230,9 @@ class CodegenAnyVal {
   llvm::Value* Eq(CodegenAnyVal* other);
 
   /// Compares this *Val to the value of 'native_ptr'. 'native_ptr' should be a pointer to
-  /// a native type, StringValue, or TimestampValue. This *Val should match 'native_ptr's
-  /// type (e.g. if this is an IntVal, 'native_ptr' should have type i32*). Returns the i1
-  /// result of the equality comparison.
+  /// a native type, e.g. StringValue, or TimestampValue. This *Val should match
+  /// 'native_ptr's type (e.g. if this is an IntVal, 'native_ptr' should have type i32*).
+  /// Returns the i1 result of the equality comparison.
   llvm::Value* EqToNativePtr(llvm::Value* native_ptr);
 
   /// Returns the i32 result of comparing this value to 'other' (similar to

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 0e2235c..2a8bb4b 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -326,15 +326,15 @@ Status LlvmCodeGen::CreateImpalaCodegen(RuntimeState* state,
   SCOPED_TIMER(codegen->prepare_module_timer_);
 
   // Get type for StringValue
-  codegen->string_val_type_ = codegen->GetType(StringValue::LLVM_CLASS_NAME);
+  codegen->string_value_type_ = codegen->GetType(StringValue::LLVM_CLASS_NAME);
 
   // Get type for TimestampValue
-  codegen->timestamp_val_type_ = codegen->GetType(TimestampValue::LLVM_CLASS_NAME);
+  codegen->timestamp_value_type_ = codegen->GetType(TimestampValue::LLVM_CLASS_NAME);
 
   // Verify size is correct
   const DataLayout& data_layout = codegen->execution_engine()->getDataLayout();
   const StructLayout* layout =
-      data_layout.getStructLayout(static_cast<StructType*>(codegen->string_val_type_));
+      data_layout.getStructLayout(static_cast<StructType*>(codegen->string_value_type_));
   if (layout->getSizeInBytes() != sizeof(StringValue)) {
     DCHECK_EQ(layout->getSizeInBytes(), sizeof(StringValue));
     return Status("Could not create llvm struct type for StringVal");
@@ -470,14 +470,17 @@ Type* LlvmCodeGen::GetType(const ColumnType& type) {
       return Type::getDoubleTy(context());
     case TYPE_STRING:
     case TYPE_VARCHAR:
-      return string_val_type_;
+      return string_value_type_;
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      // Represent this as an array of bytes.
+      return ArrayType::get(GetType(TYPE_TINYINT), type.len);
     case TYPE_CHAR:
       // IMPALA-3207: Codegen for CHAR is not yet implemented, this should not
       // be called for TYPE_CHAR.
       DCHECK(false) << "NYI";
       return NULL;
     case TYPE_TIMESTAMP:
-      return timestamp_val_type_;
+      return timestamp_value_type_;
     case TYPE_DECIMAL:
       return Type::getIntNTy(context(), type.GetByteSize() * 8);
     default:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 805f447..dca344a 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -237,7 +237,8 @@ class LlvmCodeGen {
   /// Return a pointer to pointer type for 'name' type.
   llvm::PointerType* GetPtrPtrType(const std::string& name);
 
-  /// Returns llvm type for the column type
+  /// Returns llvm type for Impala's internal representation of this column type,
+  /// i.e. the way Impala represents this type in a Tuple.
   llvm::Type* GetType(const ColumnType& type);
 
   /// Return a pointer type to 'type' (e.g. int16_t*)
@@ -474,7 +475,6 @@ class LlvmCodeGen {
   llvm::Type* bigint_type() { return GetType(TYPE_BIGINT); }
   llvm::Type* float_type() { return GetType(TYPE_FLOAT); }
   llvm::Type* double_type() { return GetType(TYPE_DOUBLE); }
-  llvm::Type* string_val_type() { return string_val_type_; }
   llvm::PointerType* ptr_type() { return ptr_type_; }
   llvm::Type* void_type() { return void_type_; }
   llvm::Type* i128_type() { return llvm::Type::getIntNTy(context(), 128); }
@@ -764,8 +764,8 @@ class LlvmCodeGen {
   /// llvm representation of a few common types.  Owned by context.
   llvm::PointerType* ptr_type_;             // int8_t*
   llvm::Type* void_type_;                   // void
-  llvm::Type* string_val_type_;             // StringValue
-  llvm::Type* timestamp_val_type_;          // TimestampValue
+  llvm::Type* string_value_type_;           // StringValue
+  llvm::Type* timestamp_value_type_;        // TimestampValue
 
   /// llvm constants to help with code gen verbosity
   llvm::Value* true_value_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index e65d9f1..c9090b9 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -818,7 +818,7 @@ Status HashTableCtx::CodegenEvalRow(LlvmCodeGen* codegen, bool build, Function**
 
     // Not null block
     builder.SetInsertPoint(not_null_block);
-    result.ToNativePtr(llvm_loc);
+    result.StoreToNativePtr(llvm_loc);
     builder.CreateBr(continue_block);
 
     // Continue block

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exec/incr-stats-util.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/incr-stats-util.cc b/be/src/exec/incr-stats-util.cc
index 7d27184..6f5e2b6 100644
--- a/be/src/exec/incr-stats-util.cc
+++ b/be/src/exec/incr-stats-util.cc
@@ -46,7 +46,6 @@ StringVal IncrementNdvFinalize(FunctionContext* ctx, const StringVal& src) {
   StringVal result_str(ctx, src.len);
   if (UNLIKELY(result_str.is_null)) return result_str;
   memcpy(result_str.ptr, src.ptr, src.len);
-  ctx->Free(src.ptr);
   return result_str;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exec/partitioned-aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-aggregation-node.cc b/be/src/exec/partitioned-aggregation-node.cc
index 16db5cc..8432bcc 100644
--- a/be/src/exec/partitioned-aggregation-node.cc
+++ b/be/src/exec/partitioned-aggregation-node.cc
@@ -1501,8 +1501,8 @@ Status PartitionedAggregationNode::QueryMaintenance(RuntimeState* state) {
 // The IR for ndv(timestamp_col), which uses the UDA interface, is:
 //
 // define void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
-//     <{ %"struct.impala::StringValue" }>* %agg_tuple,
-//     %"class.impala::TupleRow"* %row) #33 {
+//     <{ [1024 x i8] }>* %agg_tuple,
+//     %"class.impala::TupleRow"* %row) #39 {
 // entry:
 //   %dst_lowered_ptr = alloca { i64, i8* }
 //   %0 = alloca { i64, i64 }
@@ -1516,22 +1516,17 @@ Status PartitionedAggregationNode::QueryMaintenance(RuntimeState* state) {
 //   %input0 = call { i64, i64 } @GetSlotRef(
 //       %"class.impala::ScalarExprEvaluator"* %input_eval,
 //       %"class.impala::TupleRow"* %row)
-//   %dst_slot_ptr = getelementptr inbounds <{ %"struct.impala::StringValue" }>,
-//       <{ %"struct.impala::StringValue" }>* %agg_tuple, i32 0, i32 0
-//   %dst_val =
-//       load %"struct.impala::StringValue", %"struct.impala::StringValue"* %dst_slot_ptr
-//   %ptr = extractvalue %"struct.impala::StringValue" %dst_val, 0
-//   %dst = insertvalue { i64, i8* } zeroinitializer, i8* %ptr, 1
-//   %len = extractvalue %"struct.impala::StringValue" %dst_val, 1
-//   %2 = extractvalue { i64, i8* } %dst, 0
-//   %3 = zext i32 %len to i64
-//   %4 = shl i64 %3, 32
-//   %5 = and i64 %2, 4294967295
-//   %6 = or i64 %5, %4
-//   %dst1 = insertvalue { i64, i8* } %dst, i64 %6, 0
+//   %dst_slot_ptr = getelementptr inbounds <{ [1024 x i8] }>,
+//       <{ [1024 x i8] }>* %agg_tuple, i32 0, i32 0
+//   %2 = bitcast [1024 x i8]* %dst_slot_ptr to i8*
+//   %dst = insertvalue { i64, i8* } zeroinitializer, i8* %2, 1
+//   %3 = extractvalue { i64, i8* } %dst, 0
+//   %4 = and i64 %3, 4294967295
+//   %5 = or i64 %4, 4398046511104
+//   %dst1 = insertvalue { i64, i8* } %dst, i64 %5, 0
 //   %agg_fn_ctx = call %"class.impala_udf::FunctionContext"*
 //       @_ZNK6impala14AggFnEvaluator10agg_fn_ctxEv(
-//           %"class.impala::AggFnEvaluator"* %agg_fn_eval)
+//          %"class.impala::AggFnEvaluator"* %agg_fn_eval)
 //   store { i64, i64 } %input0, { i64, i64 }* %0
 //   %input_unlowered_ptr =
 //       bitcast { i64, i64 }* %0 to %"struct.impala_udf::TimestampVal"*
@@ -1543,13 +1538,6 @@ Status PartitionedAggregationNode::QueryMaintenance(RuntimeState* state) {
 //       %"struct.impala_udf::TimestampVal"* %input_unlowered_ptr,
 //       %"struct.impala_udf::StringVal"* %dst_unlowered_ptr)
 //   %anyval_result = load { i64, i8* }, { i64, i8* }* %dst_lowered_ptr
-//   %7 = extractvalue { i64, i8* } %anyval_result, 0
-//   %8 = ashr i64 %7, 32
-//   %9 = trunc i64 %8 to i32
-//   %10 = insertvalue %"struct.impala::StringValue" zeroinitializer, i32 %9, 1
-//   %11 = extractvalue { i64, i8* } %anyval_result, 1
-//   %12 = insertvalue %"struct.impala::StringValue" %10, i8* %11, 0
-//   store %"struct.impala::StringValue" %12, %"struct.impala::StringValue"* %dst_slot_ptr
 //   br label %ret
 //
 // ret:                                              ; preds = %entry
@@ -1622,36 +1610,36 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
   // 'dst_slot_ptr' points to the slot in the aggregate tuple to update.
   Value* dst_slot_ptr = builder.CreateStructGEP(
       NULL, agg_tuple_arg, slot_desc->llvm_field_idx(), "dst_slot_ptr");
-  Value* result = NULL;
-  Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
   // TODO: consider moving the following codegen logic to AggFn.
   if (agg_op == AggFn::COUNT) {
     src.CodegenBranchIfNull(&builder, ret_block);
-    if (agg_fn->is_merge()) {
-      result = builder.CreateAdd(dst_value, src.GetVal(), "count_sum");
-    } else {
-      result = builder.CreateAdd(
-          dst_value, codegen->GetIntConstant(TYPE_BIGINT, 1), "count_inc");
-    }
+    Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
+    Value* result = agg_fn->is_merge()
+        ? builder.CreateAdd(dst_value, src.GetVal(), "count_sum")
+        : builder.CreateAdd(
+            dst_value, codegen->GetIntConstant(TYPE_BIGINT, 1), "count_inc");
+    builder.CreateStore(result, dst_slot_ptr);
     DCHECK(!slot_desc->is_nullable());
   } else if ((agg_op == AggFn::MIN || agg_op == AggFn::MAX) && dst_is_numeric_or_bool) {
     bool is_min = agg_op == AggFn::MIN;
     src.CodegenBranchIfNull(&builder, ret_block);
     Function* min_max_fn = codegen->CodegenMinMax(slot_desc->type(), is_min);
+    Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
     Value* min_max_args[] = {dst_value, src.GetVal()};
-    result =
+    Value* result =
         builder.CreateCall(min_max_fn, min_max_args, is_min ? "min_value" : "max_value");
+    builder.CreateStore(result, dst_slot_ptr);
     // Dst may have been NULL, make sure to unset the NULL bit.
     DCHECK(slot_desc->is_nullable());
     slot_desc->CodegenSetNullIndicator(
         codegen, &builder, agg_tuple_arg, codegen->false_value());
   } else if (agg_op == AggFn::SUM && dst_is_int_or_float_or_bool) {
     src.CodegenBranchIfNull(&builder, ret_block);
-    if (dst_type.IsFloatingPointType()) {
-      result = builder.CreateFAdd(dst_value, src.GetVal());
-    } else {
-      result = builder.CreateAdd(dst_value, src.GetVal());
-    }
+    Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
+    Value* result = dst_type.IsFloatingPointType()
+        ? builder.CreateFAdd(dst_value, src.GetVal())
+        : builder.CreateAdd(dst_value, src.GetVal());
+    builder.CreateStore(result, dst_slot_ptr);
 
     if (slot_desc->is_nullable()) {
       slot_desc->CodegenSetNullIndicator(
@@ -1685,7 +1673,7 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
         dst.SetIsNull(slot_desc->CodegenIsNull(codegen, &builder, agg_tuple_arg));
       }
     }
-    dst.SetFromRawValue(dst_value);
+    dst.LoadFromNativePtr(dst_slot_ptr);
 
     // Get the FunctionContext object for the AggFnEvaluator.
     Function* get_agg_fn_ctx_fn =
@@ -1699,7 +1687,11 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
     CodegenAnyVal updated_dst_val;
     RETURN_IF_ERROR(CodegenCallUda(codegen, &builder, agg_fn, agg_fn_ctx_val,
         input_vals, dst, &updated_dst_val));
-    result = updated_dst_val.ToNativeValue();
+    // Copy the value back to the slot. In the FIXED_UDA_INTERMEDIATE case, the
+    // UDA function writes directly to the slot so there is nothing to copy.
+    if (dst_type.type != TYPE_FIXED_UDA_INTERMEDIATE) {
+      updated_dst_val.StoreToNativePtr(dst_slot_ptr);
+    }
 
     if (slot_desc->is_nullable() && !special_null_handling) {
       // Set NULL bit in the slot based on the return value.
@@ -1708,9 +1700,6 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
           codegen, &builder, agg_tuple_arg, result_is_null);
     }
   }
-
-  // TODO: Store to register in the loop and store once to memory at the end of the loop.
-  builder.CreateStore(result, dst_slot_ptr);
   builder.CreateBr(ret_block);
 
   builder.SetInsertPoint(ret_block);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/agg-fn-evaluator.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/agg-fn-evaluator.cc b/be/src/exprs/agg-fn-evaluator.cc
index 3a782be..f5a6098 100644
--- a/be/src/exprs/agg-fn-evaluator.cc
+++ b/be/src/exprs/agg-fn-evaluator.cc
@@ -26,6 +26,7 @@
 #include "exprs/scalar-expr.h"
 #include "exprs/scalar-expr-evaluator.h"
 #include "exprs/scalar-fn-call.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/lib-cache.h"
 #include "runtime/raw-value.h"
 #include "runtime/runtime-state.h"
@@ -225,8 +226,10 @@ void AggFnEvaluator::SetDstSlot(const AnyVal* src, const SlotDescriptor& dst_slo
           StringValue::FromStringVal(*reinterpret_cast<const StringVal*>(src));
       return;
     case TYPE_CHAR:
-      if (slot != reinterpret_cast<const StringVal*>(src)->ptr) {
-        agg_fn_ctx_->SetError("UDA should not set pointer of CHAR(N) intermediate");
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      if (UNLIKELY(slot != reinterpret_cast<const StringVal*>(src)->ptr)) {
+        agg_fn_ctx_->SetError(Substitute("UDA should not set pointer of $0 intermediate",
+              dst_slot_desc.type().DebugString()).c_str());
       }
       return;
     case TYPE_TIMESTAMP:
@@ -272,9 +275,10 @@ void AggFnEvaluator::Init(Tuple* dst) {
 
   const ColumnType& type = intermediate_type();
   const SlotDescriptor& slot_desc = intermediate_slot_desc();
-  if (type.type == TYPE_CHAR) {
-    // For type char, we want to initialize the staging_intermediate_val_ with
-    // a pointer into the tuple (the UDA should not be allocating it).
+  if (type.type == TYPE_CHAR || type.type == TYPE_FIXED_UDA_INTERMEDIATE) {
+    // The intermediate value is represented as a fixed-length buffer inline in the tuple.
+    // The aggregate function writes to this buffer directly. staging_intermediate_val_
+    // is a StringVal with a pointer to the slot and the length of the slot.
     void* slot = dst->GetSlot(slot_desc.tuple_offset());
     StringVal* sv = reinterpret_cast<StringVal*>(staging_intermediate_val_);
     sv->is_null = dst->IsNull(slot_desc.null_indicator_offset());
@@ -471,6 +475,20 @@ void AggFnEvaluator::SerializeOrFinalize(Tuple* src,
       SetDstSlot(&v, dst_slot_desc, dst);
       break;
     }
+    case TYPE_CHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE: {
+      // Serialize() or Finalize() may rewrite the data in place, but must return the
+      // same pointer.
+      typedef StringVal(*Fn)(FunctionContext*, AnyVal*);
+      StringVal v = reinterpret_cast<Fn>(fn)(
+          agg_fn_ctx_.get(), staging_intermediate_val_);
+      if (UNLIKELY(dst->GetSlot(dst_slot_desc.tuple_offset()) != v.ptr)) {
+        agg_fn_ctx_->SetError(Substitute("UDA Serialize() and Finalize() must return "
+            "same pointer as input for $0 intermediate",
+            dst_slot_desc.type().DebugString()).c_str());
+      }
+      break;
+    }
     default:
       DCHECK(false) << "NYI";
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/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 c7f0658..049344e 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -269,7 +269,11 @@ struct AvgState {
 };
 
 void AggregateFunctions::AvgInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(AvgState));
+  // avg() uses a preallocated FIXED_UDA_INTERMEDIATE intermediate value.
+  DCHECK_EQ(dst->len, sizeof(AvgState));
+  AvgState* avg = reinterpret_cast<AvgState*>(dst->ptr);
+  avg->sum = 0.0;
+  avg->count = 0;
 }
 
 template <typename T>
@@ -314,7 +318,6 @@ DoubleVal AggregateFunctions::AvgGetValue(FunctionContext* ctx, const StringVal&
 DoubleVal AggregateFunctions::AvgFinalize(FunctionContext* ctx, const StringVal& src) {
   if (UNLIKELY(src.is_null)) return DoubleVal::null();
   DoubleVal result = AvgGetValue(ctx, src);
-  ctx->Free(src.ptr);
   return result;
 }
 
@@ -366,17 +369,20 @@ TimestampVal AggregateFunctions::TimestampAvgFinalize(FunctionContext* ctx,
     const StringVal& src) {
   if (UNLIKELY(src.is_null)) return TimestampVal::null();
   TimestampVal result = TimestampAvgGetValue(ctx, src);
-  ctx->Free(src.ptr);
   return result;
 }
 
 struct DecimalAvgState {
-  DecimalVal sum; // only using val16
+  __int128_t sum_val16; // Always uses max precision decimal.
   int64_t count;
 };
 
 void AggregateFunctions::DecimalAvgInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(DecimalAvgState));
+  // avg() uses a preallocated FIXED_UDA_INTERMEDIATE intermediate value.
+  DCHECK_EQ(dst->len, sizeof(DecimalAvgState));
+  DecimalAvgState* avg = reinterpret_cast<DecimalAvgState*>(dst->ptr);
+  avg->sum_val16 = 0;
+  avg->count = 0;
 }
 
 void AggregateFunctions::DecimalAvgUpdate(FunctionContext* ctx, const DecimalVal& src,
@@ -402,13 +408,13 @@ IR_ALWAYS_INLINE void AggregateFunctions::DecimalAvgAddOrRemove(FunctionContext*
   int m = remove ? -1 : 1;
   switch (ctx->impl()->GetConstFnAttr(FunctionContextImpl::ARG_TYPE_SIZE, 0)) {
     case 4:
-      avg->sum.val16 += m * src.val4;
+      avg->sum_val16 += m * src.val4;
       break;
     case 8:
-      avg->sum.val16 += m * src.val8;
+      avg->sum_val16 += m * src.val8;
       break;
     case 16:
-      avg->sum.val16 += m * src.val16;
+      avg->sum_val16 += m * src.val16;
       break;
     default:
       DCHECK(false) << "Invalid byte size";
@@ -428,7 +434,7 @@ void AggregateFunctions::DecimalAvgMerge(FunctionContext* ctx,
   DCHECK(dst->ptr != NULL);
   DCHECK_EQ(sizeof(DecimalAvgState), dst->len);
   DecimalAvgState* dst_struct = reinterpret_cast<DecimalAvgState*>(dst->ptr);
-  dst_struct->sum.val16 += src_struct->sum.val16;
+  dst_struct->sum_val16 += src_struct->sum_val16;
   dst_struct->count += src_struct->count;
 }
 
@@ -436,7 +442,7 @@ DecimalVal AggregateFunctions::DecimalAvgGetValue(FunctionContext* ctx,
     const StringVal& src) {
   DecimalAvgState* val_struct = reinterpret_cast<DecimalAvgState*>(src.ptr);
   if (val_struct->count == 0) return DecimalVal::null();
-  Decimal16Value sum(val_struct->sum.val16);
+  Decimal16Value sum(val_struct->sum_val16);
   Decimal16Value count(val_struct->count);
 
   int output_precision =
@@ -461,7 +467,6 @@ DecimalVal AggregateFunctions::DecimalAvgFinalize(FunctionContext* ctx,
     const StringVal& src) {
   if (UNLIKELY(src.is_null)) return DecimalVal::null();
   DecimalVal result = DecimalAvgGetValue(ctx, src);
-  ctx->Free(src.ptr);
   return result;
 }
 
@@ -733,26 +738,28 @@ const static int PC_BITMAP_LENGTH = 32; // the length of each bit map
 const static float PC_THETA = 0.77351f; // the magic number to compute the final result
 const static float PC_K = -1.75f; // the magic correction for low cardinalities
 
+// Size of the distinct estimate bit map - Probabilistic Counting Algorithms for Data
+// Base Applications (Flajolet and Martin)
+//
+// The bitmap is a 64bit(1st index) x 32bit(2nd index) matrix.
+// So, the string length of 256 byte is enough.
+// The layout is:
+//   row  1: 8bit 8bit 8bit 8bit
+//   row  2: 8bit 8bit 8bit 8bit
+//   ...     ..
+//   ...     ..
+//   row 64: 8bit 8bit 8bit 8bit
+//
+// Using 32bit length, we can count up to 10^8. This will not be enough for Fact table
+// primary key, but once we approach the limit, we could interpret the result as
+// "every row is distinct".
+const static int PC_INTERMEDIATE_BYTES = NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8;
+
 void AggregateFunctions::PcInit(FunctionContext* c, StringVal* dst) {
-  // Initialize the distinct estimate bit map - Probabilistic Counting Algorithms for Data
-  // Base Applications (Flajolet and Martin)
-  //
-  // The bitmap is a 64bit(1st index) x 32bit(2nd index) matrix.
-  // So, the string length of 256 byte is enough.
-  // The layout is:
-  //   row  1: 8bit 8bit 8bit 8bit
-  //   row  2: 8bit 8bit 8bit 8bit
-  //   ...     ..
-  //   ...     ..
-  //   row 64: 8bit 8bit 8bit 8bit
-  //
-  // Using 32bit length, we can count up to 10^8. This will not be enough for Fact table
-  // primary key, but once we approach the limit, we could interpret the result as
-  // "every row is distinct".
-  //
-  // We use "string" type for DISTINCT_PC function so that we can use the string
-  // slot to hold the bitmaps.
-  AllocBuffer(c, dst, NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8);
+  // The distinctpc*() functions use a preallocated FIXED_UDA_INTERMEDIATE intermediate
+  // value.
+  DCHECK_EQ(dst->len, PC_INTERMEDIATE_BYTES);
+  memset(dst->ptr, 0, PC_INTERMEDIATE_BYTES);
 }
 
 static inline void SetDistinctEstimateBit(uint8_t* bitmap,
@@ -772,6 +779,7 @@ static inline bool GetDistinctEstimateBit(uint8_t* bitmap,
 
 template<typename T>
 void AggregateFunctions::PcUpdate(FunctionContext* c, const T& input, StringVal* dst) {
+  DCHECK_EQ(dst->len, PC_INTERMEDIATE_BYTES);
   if (input.is_null) return;
   // Core of the algorithm. This is a direct translation of the code in the paper.
   // Please see the paper for details. For simple averaging, we need to compute hash
@@ -787,6 +795,7 @@ void AggregateFunctions::PcUpdate(FunctionContext* c, const T& input, StringVal*
 
 template<typename T>
 void AggregateFunctions::PcsaUpdate(FunctionContext* c, const T& input, StringVal* dst) {
+  DCHECK_EQ(dst->len, PC_INTERMEDIATE_BYTES);
   if (input.is_null) return;
 
   // Core of the algorithm. This is a direct translation of the code in the paper.
@@ -822,12 +831,13 @@ void AggregateFunctions::PcMerge(FunctionContext* c,
     const StringVal& src, StringVal* dst) {
   DCHECK(!src.is_null);
   DCHECK(!dst->is_null);
-  DCHECK_EQ(src.len, NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8);
+  DCHECK_EQ(src.len, PC_INTERMEDIATE_BYTES);
+  DCHECK_EQ(dst->len, PC_INTERMEDIATE_BYTES);
 
   // Merge the bits
   // I think _mm_or_ps can do it, but perf doesn't really matter here. We call this only
   // once group per node.
-  for (int i = 0; i < NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8; ++i) {
+  for (int i = 0; i < PC_INTERMEDIATE_BYTES; ++i) {
     *(dst->ptr + i) |= *(src.ptr + i);
   }
 
@@ -837,9 +847,9 @@ void AggregateFunctions::PcMerge(FunctionContext* c,
            << DistinctEstimateBitMapToString(dst->ptr);
 }
 
-static double DistinceEstimateFinalize(const StringVal& src) {
+static double DistinctEstimateFinalize(const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8);
+  DCHECK_EQ(src.len, PC_INTERMEDIATE_BYTES);
   VLOG_ROW << "FinalizeEstimateSlot Bit map:\n"
            << DistinctEstimateBitMapToString(src.ptr);
 
@@ -847,7 +857,7 @@ static double DistinceEstimateFinalize(const StringVal& src) {
   // distinct rows. We're overwriting the result in the same string buffer we've
   // allocated.
   bool is_empty = true;
-  for (int i = 0; i < NUM_PC_BITMAPS * PC_BITMAP_LENGTH / 8; ++i) {
+  for (int i = 0; i < PC_INTERMEDIATE_BYTES; ++i) {
     if (src.ptr[i] != 0) {
       is_empty = false;
       break;
@@ -884,16 +894,14 @@ static double DistinceEstimateFinalize(const StringVal& src) {
 
 BigIntVal AggregateFunctions::PcFinalize(FunctionContext* c, const StringVal& src) {
   if (UNLIKELY(src.is_null)) return BigIntVal::null();
-  double estimate = DistinceEstimateFinalize(src);
-  c->Free(src.ptr);
+  double estimate = DistinctEstimateFinalize(src);
   return static_cast<int64_t>(estimate);
 }
 
 BigIntVal AggregateFunctions::PcsaFinalize(FunctionContext* c, const StringVal& src) {
   if (UNLIKELY(src.is_null)) return BigIntVal::null();
   // When using stochastic averaging, the result has to be multiplied by NUM_PC_BITMAPS.
-  double estimate = DistinceEstimateFinalize(src) * NUM_PC_BITMAPS;
-  c->Free(src.ptr);
+  double estimate = DistinctEstimateFinalize(src) * NUM_PC_BITMAPS;
   return static_cast<int64_t>(estimate);
 }
 
@@ -1333,7 +1341,9 @@ T AggregateFunctions::AppxMedianFinalize(FunctionContext* ctx, const StringVal&
 }
 
 void AggregateFunctions::HllInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, HLL_LEN);
+  // The HLL functions use a preallocated FIXED_UDA_INTERMEDIATE intermediate value.
+  DCHECK_EQ(dst->len, HLL_LEN);
+  memset(dst->ptr, 0, HLL_LEN);
 }
 
 template <typename T>
@@ -1424,7 +1434,6 @@ uint64_t AggregateFunctions::HllFinalEstimate(const uint8_t* buckets,
 BigIntVal AggregateFunctions::HllFinalize(FunctionContext* ctx, const StringVal& src) {
   if (UNLIKELY(src.is_null)) return BigIntVal::null();
   uint64_t estimate = HllFinalEstimate(src.ptr, src.len);
-  ctx->Free(src.ptr);
   return estimate;
 }
 
@@ -1447,7 +1456,8 @@ static double ComputeKnuthVariance(const KnuthVarianceState& state, bool pop) {
 }
 
 void AggregateFunctions::KnuthVarInit(FunctionContext* ctx, StringVal* dst) {
-  dst->is_null = false;
+  // The Knuth variance functions use a preallocated FIXED_UDA_INTERMEDIATE intermediate
+  // value.
   DCHECK_EQ(dst->len, sizeof(KnuthVarianceState));
   memset(dst->ptr, 0, dst->len);
 }
@@ -1529,11 +1539,8 @@ struct RankState {
 };
 
 void AggregateFunctions::RankInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(RankState));
-  if (UNLIKELY(dst->is_null)) {
-    DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
-    return;
-  }
+  // The rank functions use a preallocated FIXED_UDA_INTERMEDIATE intermediate value.
+  DCHECK_EQ(dst->len, sizeof(RankState));
   *reinterpret_cast<RankState*>(dst->ptr) = RankState();
 }
 
@@ -1581,7 +1588,6 @@ BigIntVal AggregateFunctions::RankFinalize(FunctionContext* ctx,
   DCHECK_EQ(src_val.len, sizeof(RankState));
   RankState* state = reinterpret_cast<RankState*>(src_val.ptr);
   int64_t result = state->rank;
-  ctx->Free(src_val.ptr);
   return BigIntVal(result);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/anyval-util.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/anyval-util.cc b/be/src/exprs/anyval-util.cc
index c49cdb3..b46dfdf 100644
--- a/be/src/exprs/anyval-util.cc
+++ b/be/src/exprs/anyval-util.cc
@@ -81,6 +81,10 @@ FunctionContext::TypeDesc AnyValUtil::ColumnTypeToTypeDesc(const ColumnType& typ
       out.type = FunctionContext::TYPE_FIXED_BUFFER;
       out.len = type.len;
       break;
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      out.type = FunctionContext::TYPE_FIXED_UDA_INTERMEDIATE;
+      out.len = type.len;
+      break;
     case TYPE_DECIMAL:
       out.type = FunctionContext::TYPE_DECIMAL;
       out.precision = type.precision;
@@ -123,6 +127,8 @@ ColumnType AnyValUtil::TypeDescToColumnType(const FunctionContext::TypeDesc& typ
       return ColumnType::CreateDecimalType(type.precision, type.scale);
     case FunctionContext::TYPE_FIXED_BUFFER:
       return ColumnType::CreateCharType(type.len);
+    case FunctionContext::TYPE_FIXED_UDA_INTERMEDIATE:
+      return ColumnType::CreateFixedUdaIntermediateType(type.len);
     case FunctionContext::TYPE_VARCHAR:
       return ColumnType::CreateVarcharType(type.len);
     default:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/anyval-util.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/anyval-util.h b/be/src/exprs/anyval-util.h
index c908ced..d83acee 100644
--- a/be/src/exprs/anyval-util.h
+++ b/be/src/exprs/anyval-util.h
@@ -188,6 +188,7 @@ class AnyValUtil {
       case TYPE_STRING:
       case TYPE_VARCHAR:
       case TYPE_CHAR:
+      case TYPE_FIXED_UDA_INTERMEDIATE:
         return sizeof(StringVal);
       case TYPE_TIMESTAMP: return sizeof(TimestampVal);
       case TYPE_DECIMAL: return sizeof(DecimalVal);
@@ -210,6 +211,7 @@ class AnyValUtil {
       case TYPE_STRING:
       case TYPE_VARCHAR:
       case TYPE_CHAR:
+      case TYPE_FIXED_UDA_INTERMEDIATE:
         return alignof(StringVal);
       case TYPE_TIMESTAMP: return alignof(TimestampVal);
       case TYPE_DECIMAL: return alignof(DecimalVal);
@@ -287,7 +289,8 @@ class AnyValUtil {
           DCHECK_LE(sv->len, type.len);
         }
         return;
-      case TYPE_CHAR: {
+      case TYPE_CHAR:
+      case TYPE_FIXED_UDA_INTERMEDIATE: {
         StringVal* sv = reinterpret_cast<StringVal*>(dst);
         sv->ptr = const_cast<uint8_t*>(reinterpret_cast<const uint8_t*>(slot));
         sv->len = type.len;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/scalar-expr-evaluator.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-expr-evaluator.cc b/be/src/exprs/scalar-expr-evaluator.cc
index 6a401ef..b2e0aa6 100644
--- a/be/src/exprs/scalar-expr-evaluator.cc
+++ b/be/src/exprs/scalar-expr-evaluator.cc
@@ -320,7 +320,8 @@ void* ScalarExprEvaluator::GetValue(const ScalarExpr& expr, const TupleRow* row)
       result_.string_val.len = v.len;
       return &result_.string_val;
     }
-    case TYPE_CHAR: {
+    case TYPE_CHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE: {
       impala_udf::StringVal v = expr.GetStringVal(this, row);
       if (v.is_null) return nullptr;
       result_.string_val.ptr = reinterpret_cast<char*>(v.ptr);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/slot-ref.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/slot-ref.cc b/be/src/exprs/slot-ref.cc
index 5c2d552..9f1a77d 100644
--- a/be/src/exprs/slot-ref.cc
+++ b/be/src/exprs/slot-ref.cc
@@ -251,7 +251,11 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
     ptr = builder.CreateLoad(ptr_ptr, "ptr");
     Value* len_ptr = builder.CreateStructGEP(NULL, val_ptr, 1, "len_ptr");
     len = builder.CreateLoad(len_ptr, "len");
-  } else if (type() == TYPE_TIMESTAMP) {
+  } else if (type_.type == TYPE_FIXED_UDA_INTERMEDIATE) {
+    // ptr and len are the slot and its fixed length.
+    ptr = builder.CreateBitCast(val_ptr, codegen->ptr_type());
+    len = codegen->GetIntConstant(TYPE_INT, type_.len);
+  } else if (type_.type == TYPE_TIMESTAMP) {
     Value* time_of_day_ptr = builder.CreateStructGEP(NULL, val_ptr, 0, "time_of_day_ptr");
     // Cast boost::posix_time::time_duration to i64
     Value* time_of_day_cast =
@@ -280,7 +284,7 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   // *Val. The optimizer does a better job when there is a phi node for each value, rather
   // than having get_slot_block generate an AnyVal and having a single phi node over that.
   // TODO: revisit this code, can possibly be simplified
-  if (type().IsVarLenStringType()) {
+  if (type_.IsVarLenStringType() || type_.type == TYPE_FIXED_UDA_INTERMEDIATE) {
     DCHECK(ptr != NULL);
     DCHECK(len != NULL);
     PHINode* ptr_phi = builder.CreatePHI(ptr->getType(), 2, "ptr_phi");
@@ -309,7 +313,7 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
     result.SetPtr(ptr_phi);
     result.SetLen(len_phi);
     builder.CreateRet(result.GetLoweredValue());
-  } else if (type() == TYPE_TIMESTAMP) {
+  } else if (type_.type == TYPE_TIMESTAMP) {
     DCHECK(time_of_day != NULL);
     DCHECK(date != NULL);
     PHINode* time_of_day_phi =
@@ -423,11 +427,11 @@ DoubleVal SlotRef::GetDoubleVal(
 
 StringVal SlotRef::GetStringVal(
     ScalarExprEvaluator* eval, const TupleRow* row) const {
-  DCHECK(type_.IsStringType());
+  DCHECK(type_.IsStringType() || type_.type == TYPE_FIXED_UDA_INTERMEDIATE);
   Tuple* t = row->GetTuple(tuple_idx_);
   if (t == NULL || t->IsNull(null_indicator_offset_)) return StringVal::null();
   StringVal result;
-  if (type_.type == TYPE_CHAR) {
+  if (type_.type == TYPE_CHAR || type_.type == TYPE_FIXED_UDA_INTERMEDIATE) {
     result.ptr = reinterpret_cast<uint8_t*>(t->GetSlot(slot_offset_));
     result.len = type_.len;
   } else {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/exprs/utility-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/utility-functions-ir.cc b/be/src/exprs/utility-functions-ir.cc
index 263c4d4..18c4267 100644
--- a/be/src/exprs/utility-functions-ir.cc
+++ b/be/src/exprs/utility-functions-ir.cc
@@ -133,9 +133,10 @@ StringVal UtilityFunctions::TypeOf(FunctionContext* ctx, const T& /*input_val*/)
     case TYPE_DECIMAL:
       return AnyValUtil::FromString(ctx, Substitute("$0($1,$2)", type_string,
           type_desc.precision, type_desc.scale));
-    // Show length of CHAR and VARCHAR.
+    // Show types parameterised by length.
     case TYPE_CHAR:
     case TYPE_VARCHAR:
+    case TYPE_FIXED_UDA_INTERMEDIATE:
       return AnyValUtil::FromString(ctx, Substitute("$0($1)", type_string,
           type_desc.len));
     default:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/runtime/raw-value.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/raw-value.cc b/be/src/runtime/raw-value.cc
index bdcbb11..d7f4fd5 100644
--- a/be/src/runtime/raw-value.cc
+++ b/be/src/runtime/raw-value.cc
@@ -154,7 +154,7 @@ void RawValue::Write(const void* value, void* dst, const ColumnType& type,
       if (pool != NULL) {
         // Note: if this changes to TryAllocate(), CodegenAnyVal::WriteToSlot() will need
         // to reflect this change as well (the codegen'd Allocate() call is actually
-        // generated in CodegenAnyVal::ToNativeValue()).
+        // generated in CodegenAnyVal::StoreToNativePtr()).
         dest->ptr = reinterpret_cast<char*>(pool->Allocate(dest->len));
         memcpy(dest->ptr, src->ptr, dest->len);
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/runtime/types.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/types.cc b/be/src/runtime/types.cc
index 2de50e6..1b21fe8 100644
--- a/be/src/runtime/types.cc
+++ b/be/src/runtime/types.cc
@@ -49,7 +49,8 @@ ColumnType::ColumnType(const std::vector<TTypeNode>& types, int* idx)
       DCHECK(node.__isset.scalar_type);
       const TScalarType scalar_type = node.scalar_type;
       type = ThriftToType(scalar_type.type);
-      if (type == TYPE_CHAR || type == TYPE_VARCHAR) {
+      if (type == TYPE_CHAR || type == TYPE_VARCHAR
+          || type == TYPE_FIXED_UDA_INTERMEDIATE) {
         DCHECK(scalar_type.__isset.len);
         len = scalar_type.len;
       } else if (type == TYPE_DECIMAL) {
@@ -108,6 +109,7 @@ PrimitiveType ThriftToType(TPrimitiveType::type ttype) {
     case TPrimitiveType::BINARY: return TYPE_BINARY;
     case TPrimitiveType::DECIMAL: return TYPE_DECIMAL;
     case TPrimitiveType::CHAR: return TYPE_CHAR;
+    case TPrimitiveType::FIXED_UDA_INTERMEDIATE: return TYPE_FIXED_UDA_INTERMEDIATE;
     default: return INVALID_TYPE;
   }
 }
@@ -131,6 +133,7 @@ TPrimitiveType::type ToThrift(PrimitiveType ptype) {
     case TYPE_BINARY: return TPrimitiveType::BINARY;
     case TYPE_DECIMAL: return TPrimitiveType::DECIMAL;
     case TYPE_CHAR: return TPrimitiveType::CHAR;
+    case TYPE_FIXED_UDA_INTERMEDIATE: return TPrimitiveType::FIXED_UDA_INTERMEDIATE;
     case TYPE_STRUCT:
     case TYPE_ARRAY:
     case TYPE_MAP:
@@ -158,6 +161,7 @@ string TypeToString(PrimitiveType t) {
     case TYPE_BINARY: return "BINARY";
     case TYPE_DECIMAL: return "DECIMAL";
     case TYPE_CHAR: return "CHAR";
+    case TYPE_FIXED_UDA_INTERMEDIATE: return "FIXED_UDA_INTERMEDIATE";
     case TYPE_STRUCT: return "STRUCT";
     case TYPE_ARRAY: return "ARRAY";
     case TYPE_MAP: return "MAP";
@@ -188,6 +192,10 @@ string TypeToOdbcString(PrimitiveType t) {
     case TYPE_STRUCT: return "struct";
     case TYPE_ARRAY: return "array";
     case TYPE_MAP: return "map";
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      // This type is not exposed to clients and should not be returned.
+      DCHECK(false);
+      break;
   };
   return "unknown";
 }
@@ -217,7 +225,8 @@ void ColumnType::ToThrift(TColumnType* thrift_type) const {
     node.__set_scalar_type(TScalarType());
     TScalarType& scalar_type = node.scalar_type;
     scalar_type.__set_type(impala::ToThrift(type));
-    if (type == TYPE_CHAR || type == TYPE_VARCHAR) {
+    if (type == TYPE_CHAR || type == TYPE_VARCHAR
+        || type == TYPE_FIXED_UDA_INTERMEDIATE) {
       DCHECK_NE(len, -1);
       scalar_type.__set_len(len);
     } else if (type == TYPE_DECIMAL) {
@@ -293,7 +302,8 @@ TTypeEntry ColumnType::ToHs2Type() const {
       break;
     }
     default:
-      // HiveServer2 does not have a type for invalid, date and datetime.
+      // HiveServer2 does not have a type for invalid, date, datetime or
+      // fixed_uda_intermediate.
       DCHECK(false) << "bad TypeToTValueType() type: " << DebugString();
       type_entry.__set_type(TTypeId::STRING_TYPE);
   };
@@ -315,6 +325,9 @@ string ColumnType::DebugString() const {
     case TYPE_VARCHAR:
       ss << "VARCHAR(" << len << ")";
       return ss.str();
+    case TYPE_FIXED_UDA_INTERMEDIATE:
+      ss << "FIXED_UDA_INTERMEDIATE(" << len << ")";
+      return ss.str();
     default:
       return TypeToString(type);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/runtime/types.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h
index 4961db2..1509c1f 100644
--- a/be/src/runtime/types.h
+++ b/be/src/runtime/types.h
@@ -50,12 +50,9 @@ enum PrimitiveType {
   TYPE_DATETIME,    // Not implemented
   TYPE_BINARY,      // Not implemented
   TYPE_DECIMAL,
-
-  /// This is minimally supported currently. It can't be returned to the user or
-  /// parsed from scan nodes. It can be returned from exprs and must be consumable
-  /// by exprs.
   TYPE_CHAR,
   TYPE_VARCHAR,
+  TYPE_FIXED_UDA_INTERMEDIATE,
 
   TYPE_STRUCT,
   TYPE_ARRAY,
@@ -72,7 +69,7 @@ std::string TypeToOdbcString(PrimitiveType t);
 // TODO for 2.3: rename to TypeDescriptor
 struct ColumnType {
   PrimitiveType type;
-  /// Only set if type == TYPE_CHAR or type == TYPE_VARCHAR
+  /// Only set if type one of TYPE_CHAR, TYPE_VARCHAR, TYPE_FIXED_UDA_INTERMEDIATE.
   int len;
   static const int MAX_VARCHAR_LENGTH = (1 << 16) - 1; // 65535
   static const int MAX_CHAR_LENGTH = (1 << 8) - 1; // 255
@@ -106,6 +103,7 @@ struct ColumnType {
     DCHECK_NE(type, TYPE_STRUCT);
     DCHECK_NE(type, TYPE_ARRAY);
     DCHECK_NE(type, TYPE_MAP);
+    DCHECK_NE(type, TYPE_FIXED_UDA_INTERMEDIATE);
   }
 
   static ColumnType CreateCharType(int len) {
@@ -126,6 +124,14 @@ struct ColumnType {
     return ret;
   }
 
+  static ColumnType CreateFixedUdaIntermediateType(int len) {
+    DCHECK_GE(len, 1);
+    ColumnType ret;
+    ret.type = TYPE_FIXED_UDA_INTERMEDIATE;
+    ret.len = len;
+    return ret;
+  }
+
   static bool ValidateDecimalParams(int precision, int scale) {
     return precision >= 1 && precision <= MAX_PRECISION && scale >= 0
         && scale <= MAX_SCALE && scale <= precision;
@@ -163,7 +169,7 @@ struct ColumnType {
   bool operator==(const ColumnType& o) const {
     if (type != o.type) return false;
     if (children != o.children) return false;
-    if (type == TYPE_CHAR) return len == o.len;
+    if (type == TYPE_CHAR || type == TYPE_FIXED_UDA_INTERMEDIATE) return len == o.len;
     if (type == TYPE_DECIMAL) return precision == o.precision && scale == o.scale;
     return true;
   }
@@ -222,6 +228,7 @@ struct ColumnType {
       case TYPE_VARCHAR:
         return 0;
       case TYPE_CHAR:
+      case TYPE_FIXED_UDA_INTERMEDIATE:
         return len;
       case TYPE_NULL:
       case TYPE_BOOLEAN:
@@ -255,6 +262,7 @@ struct ColumnType {
       case TYPE_VARCHAR:
         return 16;
       case TYPE_CHAR:
+      case TYPE_FIXED_UDA_INTERMEDIATE:
         return len;
       case TYPE_ARRAY:
       case TYPE_MAP:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/testutil/test-udas.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/test-udas.cc b/be/src/testutil/test-udas.cc
index 806a971..c77ae9f 100644
--- a/be/src/testutil/test-udas.cc
+++ b/be/src/testutil/test-udas.cc
@@ -279,3 +279,39 @@ void CountNullsUpdate(FunctionContext* context, const BigIntVal& val, BigIntVal*
 void CountNullsMerge(FunctionContext* context, const BigIntVal& src, BigIntVal* dst) {
   dst->val += src.val;
 }
+
+// Defines AggCharIntermediate(INT) returns INT with CHAR(10) intermediate.
+// The function computes the sum of the input.
+static void ValidateCharIntermediateFunctionContext(const FunctionContext* context) {
+  assert(context->GetNumArgs() == 1);
+  assert(context->GetArgType(0)->type == FunctionContext::TYPE_INT);
+  assert(context->GetIntermediateType().type == FunctionContext::TYPE_FIXED_BUFFER);
+  assert(context->GetIntermediateType().len == 10);
+  assert(context->GetReturnType().type == FunctionContext::TYPE_INT);
+}
+void AggCharIntermediateInit(FunctionContext* context, StringVal* dst) {
+  ValidateCharIntermediateFunctionContext(context);
+  assert(dst->len == 10);
+  memset(dst->ptr, 0, 10);
+}
+void AggCharIntermediateUpdate(
+    FunctionContext* context, const IntVal& val, StringVal* dst) {
+  ValidateCharIntermediateFunctionContext(context);
+  assert(dst->len == 10);
+  int* dst_val = reinterpret_cast<int*>(dst->ptr);
+  if (!val.is_null) *dst_val += val.val;
+}
+void AggCharIntermediateMerge(FunctionContext* context, const StringVal& src, StringVal* dst) {
+  ValidateCharIntermediateFunctionContext(context);
+  int* dst_val = reinterpret_cast<int*>(dst->ptr);
+  *dst_val += *reinterpret_cast<int*>(src.ptr);
+}
+StringVal AggCharIntermediateSerialize(FunctionContext* context, const StringVal& in) {
+  ValidateCharIntermediateFunctionContext(context);
+  return in;
+}
+IntVal AggCharIntermediateFinalize(FunctionContext* context, const StringVal& src) {
+  ValidateCharIntermediateFunctionContext(context);
+  return IntVal(*reinterpret_cast<int*>(src.ptr));
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/be/src/udf/udf.h
----------------------------------------------------------------------
diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h
index abec935..9d77601 100644
--- a/be/src/udf/udf.h
+++ b/be/src/udf/udf.h
@@ -77,9 +77,12 @@ class FunctionContext {
     TYPE_DOUBLE,
     TYPE_TIMESTAMP,
     TYPE_STRING,
+    // Not used - maps to CHAR(N), which is not supported for UDFs and UDAs.
     TYPE_FIXED_BUFFER,
     TYPE_DECIMAL,
-    TYPE_VARCHAR
+    TYPE_VARCHAR,
+    // A fixed-size buffer, passed as a StringVal.
+    TYPE_FIXED_UDA_INTERMEDIATE
   };
 
   struct TypeDesc {
@@ -89,7 +92,8 @@ class FunctionContext {
     int precision;
     int scale;
 
-    /// Only valid if type == TYPE_FIXED_BUFFER || type == TYPE_VARCHAR
+    /// Only valid if type is one of TYPE_FIXED_BUFFER, TYPE_FIXED_UDA_INTERMEDIATE or
+    /// TYPE_VARCHAR.
     int len;
   };
 
@@ -337,10 +341,8 @@ typedef void (*UdfClose)(FunctionContext* context,
 /// The UDA is registered with three types: the result type, the input type and
 /// the intermediate type.
 ///
-/// If the UDA needs a fixed byte width intermediate buffer, the type should be
-/// TYPE_FIXED_BUFFER and Impala will allocate the buffer. If the UDA needs an unknown
-/// sized buffer, it should use TYPE_STRING and allocate it from the FunctionContext
-/// manually.
+/// If the UDA needs a variable-sized buffer, it should use TYPE_STRING and allocate it
+/// from the FunctionContext manually.
 /// For UDAs that need a complex data structure as the intermediate state, the
 /// intermediate type should be string and the UDA can cast the ptr to the structure
 /// it is using.
@@ -571,6 +573,7 @@ struct TimestampVal : public AnyVal {
   bool operator!=(const TimestampVal& other) const { return !(*this == other); }
 };
 
+/// A String value represented as a buffer + length.
 /// Note: there is a difference between a NULL string (is_null == true) and an
 /// empty string (len == 0).
 struct StringVal : public AnyVal {
@@ -579,7 +582,12 @@ struct StringVal : public AnyVal {
   // in case of overflow.
   static const unsigned MAX_LENGTH = (1 << 30);
 
+  // The length of the string buffer in bytes.
   int len;
+
+  // Pointer to the start of the string buffer. The buffer is not aligned and is not
+  // null-terminated. Functions must not read or write past the end of the buffer.
+  // I.e.  accessing ptr[i] where i >= len is invalid.
   uint8_t* ptr;
 
   /// Construct a StringVal from ptr/len. Note: this does not make a copy of ptr

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/common/thrift/Types.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Types.thrift b/common/thrift/Types.thrift
index b271dd9..280c744 100644
--- a/common/thrift/Types.thrift
+++ b/common/thrift/Types.thrift
@@ -42,12 +42,11 @@ enum TPrimitiveType {
   DATETIME,
   TIMESTAMP,
   STRING,
-  // Unsupported types
-  BINARY,
+  BINARY, // Unsupported
   DECIMAL,
-  // CHAR(n). Currently only supported in UDAs
   CHAR,
-  VARCHAR
+  VARCHAR,
+  FIXED_UDA_INTERMEDIATE,
 }
 
 enum TTypeNodeType {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
index 3eba678..07699d3 100644
--- a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
@@ -36,6 +36,26 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
 public class BuiltinsDb extends Db {
+  // Size in bytes of AvgState used for integer, floating point, and timestamp avg().
+  private static final int AVG_INTERMEDIATE_SIZE = 16;
+
+  // Size in bytes of DecimalAvgState used for decimal avg().
+  private static final int DECIMAL_AVG_INTERMEDIATE_SIZE = 32;
+
+  // Size in bytes of KnuthVarianceState used for stddev(), variance(), etc.
+  private static final int STDDEV_INTERMEDIATE_SIZE = 24;
+
+  // Size in bytes of probabilistic counting bitmap, used for distinctpc(), etc.
+  // Must match PC_INTERMEDIATE_BYTES in aggregate-functions-ir.cc.
+  private static final int PC_INTERMEDIATE_SIZE = 256;
+
+  // Size in bytes of Hyperloglog intermediate value used for ndv().
+  // Must match HLL_LEN in aggregate-functions-ir.cc.
+  private static final int HLL_INTERMEDIATE_SIZE = 1024;
+
+  // Size in bytes of RankState used for rank() and dense_rank().
+  private static final int RANK_INTERMEDIATE_SIZE = 16;
+
   public BuiltinsDb(String name, Catalog catalog) {
     super(name, catalog, createMetastoreDb(name));
     setIsSystemDb(true);
@@ -748,50 +768,54 @@ public class BuiltinsDb extends Db {
           false, false, true));
 
       // NDV
-      // TODO: this needs to switch to CHAR(64) as the intermediate type
+      Type hllIntermediateType =
+          ScalarType.createFixedUdaIntermediateType(HLL_INTERMEDIATE_SIZE);
       db.addBuiltin(AggregateFunction.createBuiltin(db, "ndv",
-          Lists.newArrayList(t), Type.BIGINT, Type.STRING,
+          Lists.newArrayList(t), Type.BIGINT, hllIntermediateType,
           prefix + "7HllInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
           prefix + HLL_UPDATE_SYMBOL.get(t),
           prefix + "8HllMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-          stringValSerializeOrFinalize,
+          null,
           prefix + "11HllFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
           true, false, true));
 
       db.addBuiltin(AggregateFunction.createBuiltin(db, "ndv_no_finalize",
-          Lists.newArrayList(t), Type.STRING, Type.STRING,
+          Lists.newArrayList(t), Type.STRING, hllIntermediateType,
           prefix + "7HllInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
           prefix + HLL_UPDATE_SYMBOL.get(t),
           prefix + "8HllMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-          stringValSerializeOrFinalize,
+          null,
           "_Z20IncrementNdvFinalizePN10impala_udf15FunctionContextERKNS_9StringValE",
           true, false, true));
 
+
+      Type pcIntermediateType =
+          ScalarType.createFixedUdaIntermediateType(PC_INTERMEDIATE_SIZE);
       // distinctpc
-      // TODO: this needs to switch to CHAR(64) as the intermediate type
       db.addBuiltin(AggregateFunction.createBuiltin(db, "distinctpc",
-          Lists.newArrayList(t), Type.BIGINT, Type.STRING,
+          Lists.newArrayList(t), Type.BIGINT, pcIntermediateType,
           prefix + "6PcInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
           prefix + PC_UPDATE_SYMBOL.get(t),
           prefix + "7PcMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-          stringValSerializeOrFinalize,
+          null,
           prefix + "10PcFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
           false, false, true));
 
       // distinctpcsa
-      // TODO: this needs to switch to CHAR(64) as the intermediate type
       db.addBuiltin(AggregateFunction.createBuiltin(db, "distinctpcsa",
-          Lists.newArrayList(t), Type.BIGINT, Type.STRING,
+          Lists.newArrayList(t), Type.BIGINT, pcIntermediateType,
           prefix + "6PcInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
           prefix + PCSA_UPDATE_SYMBOL.get(t),
           prefix + "7PcMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-          stringValSerializeOrFinalize,
+          null,
           prefix + "12PcsaFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
           false, false, true));
 
       if (STDDEV_UPDATE_SYMBOL.containsKey(t)) {
+        Type stddevIntermediateType =
+            ScalarType.createFixedUdaIntermediateType(STDDEV_INTERMEDIATE_SIZE);
         db.addBuiltin(AggregateFunction.createBuiltin(db, "stddev",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -799,7 +823,7 @@ public class BuiltinsDb extends Db {
             prefix + "19KnuthStddevFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "stddev_samp",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -807,7 +831,7 @@ public class BuiltinsDb extends Db {
             prefix + "19KnuthStddevFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "stddev_pop",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -815,7 +839,7 @@ public class BuiltinsDb extends Db {
             prefix + "22KnuthStddevPopFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "variance",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -823,7 +847,7 @@ public class BuiltinsDb extends Db {
             prefix + "16KnuthVarFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "variance_samp",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -831,7 +855,7 @@ public class BuiltinsDb extends Db {
             prefix + "16KnuthVarFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "var_samp",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -839,7 +863,7 @@ public class BuiltinsDb extends Db {
             prefix + "16KnuthVarFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "variance_pop",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -847,7 +871,7 @@ public class BuiltinsDb extends Db {
             prefix + "19KnuthVarPopFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
             false, false, false));
         db.addBuiltin(AggregateFunction.createBuiltin(db, "var_pop",
-            Lists.newArrayList(t), Type.DOUBLE, ScalarType.createCharType(24),
+            Lists.newArrayList(t), Type.DOUBLE, stddevIntermediateType,
             prefix + "12KnuthVarInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
             prefix + STDDEV_UPDATE_SYMBOL.get(t),
             prefix + "13KnuthVarMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -891,44 +915,47 @@ public class BuiltinsDb extends Db {
         null, false, true, true));
 
     // Avg
-    // TODO: switch to CHAR(sizeof(AvgIntermediateType) when that becomes available
+    Type avgIntermediateType =
+        ScalarType.createFixedUdaIntermediateType(AVG_INTERMEDIATE_SIZE);
+    Type decimalAvgIntermediateType =
+        ScalarType.createFixedUdaIntermediateType(DECIMAL_AVG_INTERMEDIATE_SIZE);
     db.addBuiltin(AggregateFunction.createBuiltin(db, "avg",
-        Lists.<Type>newArrayList(Type.BIGINT), Type.DOUBLE, Type.STRING,
+        Lists.<Type>newArrayList(Type.BIGINT), Type.DOUBLE, avgIntermediateType,
         prefix + "7AvgInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "9AvgUpdateIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE",
         prefix + "8AvgMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-        stringValSerializeOrFinalize,
+        null,
         prefix + "11AvgGetValueEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         prefix + "9AvgRemoveIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE",
         prefix + "11AvgFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         false, true, false));
     db.addBuiltin(AggregateFunction.createBuiltin(db, "avg",
-        Lists.<Type>newArrayList(Type.DOUBLE), Type.DOUBLE, Type.STRING,
+        Lists.<Type>newArrayList(Type.DOUBLE), Type.DOUBLE, avgIntermediateType,
         prefix + "7AvgInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "9AvgUpdateIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE",
         prefix + "8AvgMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-        stringValSerializeOrFinalize,
+        null,
         prefix + "11AvgGetValueEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         prefix + "9AvgRemoveIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE",
         prefix + "11AvgFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         false, true, false));
     db.addBuiltin(AggregateFunction.createBuiltin(db, "avg",
-        Lists.<Type>newArrayList(Type.DECIMAL), Type.DECIMAL, Type.STRING,
+        Lists.<Type>newArrayList(Type.DECIMAL), Type.DECIMAL, decimalAvgIntermediateType,
         prefix + "14DecimalAvgInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "16DecimalAvgUpdateEPN10impala_udf15FunctionContextERKNS1_10DecimalValEPNS1_9StringValE",
         prefix + "15DecimalAvgMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-        stringValSerializeOrFinalize,
+        null,
         prefix + "18DecimalAvgGetValueEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         prefix + "16DecimalAvgRemoveEPN10impala_udf15FunctionContextERKNS1_10DecimalValEPNS1_9StringValE",
         prefix + "18DecimalAvgFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         false, true, false));
     // Avg(Timestamp)
     db.addBuiltin(AggregateFunction.createBuiltin(db, "avg",
-        Lists.<Type>newArrayList(Type.TIMESTAMP), Type.TIMESTAMP, Type.STRING,
+        Lists.<Type>newArrayList(Type.TIMESTAMP), Type.TIMESTAMP, avgIntermediateType,
         prefix + "7AvgInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "18TimestampAvgUpdateEPN10impala_udf15FunctionContextERKNS1_12TimestampValEPNS1_9StringValE",
         prefix + "8AvgMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
-        stringValSerializeOrFinalize,
+        null,
         prefix + "20TimestampAvgGetValueEPN10impala_udf15FunctionContextERKNS1_9StringValE",
         prefix + "18TimestampAvgRemoveEPN10impala_udf15FunctionContextERKNS1_12TimestampValEPNS1_9StringValE",
         prefix + "20TimestampAvgFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
@@ -960,8 +987,10 @@ public class BuiltinsDb extends Db {
 
     // analytic functions
     // Rank
+    Type rankIntermediateType =
+        ScalarType.createFixedUdaIntermediateType(RANK_INTERMEDIATE_SIZE);
     db.addBuiltin(AggregateFunction.createAnalyticBuiltin(db, "rank",
-        Lists.<Type>newArrayList(), Type.BIGINT, Type.STRING,
+        Lists.<Type>newArrayList(), Type.BIGINT, rankIntermediateType,
         prefix + "8RankInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "10RankUpdateEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         null,
@@ -969,7 +998,7 @@ public class BuiltinsDb extends Db {
         prefix + "12RankFinalizeEPN10impala_udf15FunctionContextERNS1_9StringValE"));
     // Dense rank
     db.addBuiltin(AggregateFunction.createAnalyticBuiltin(db, "dense_rank",
-        Lists.<Type>newArrayList(), Type.BIGINT, Type.STRING,
+        Lists.<Type>newArrayList(), Type.BIGINT, rankIntermediateType,
         prefix + "8RankInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         prefix + "15DenseRankUpdateEPN10impala_udf15FunctionContextEPNS1_9StringValE",
         null,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/fe/src/main/java/org/apache/impala/catalog/Function.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Function.java b/fe/src/main/java/org/apache/impala/catalog/Function.java
index 2f0859f..80316a6 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Function.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Function.java
@@ -451,6 +451,8 @@ public class Function implements CatalogObject {
     case STRING:
     case VARCHAR:
     case CHAR:
+    case FIXED_UDA_INTERMEDIATE:
+      // These types are marshaled into a StringVal.
       return "StringVal";
     case TIMESTAMP:
       return "TimestampVal";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/852e1bb7/fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java b/fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
index 7397734..e09fd7b 100644
--- a/fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
+++ b/fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
@@ -50,7 +50,13 @@ public enum PrimitiveType {
   DECIMAL("DECIMAL", 16, TPrimitiveType.DECIMAL),
 
   // Fixed length char array.
-  CHAR("CHAR", -1, TPrimitiveType.CHAR);
+  CHAR("CHAR", -1, TPrimitiveType.CHAR),
+
+  // Fixed length binary array, stored inline in the tuple. Currently only used
+  // internally for intermediate results of builtin aggregate functions. Not exposed
+  // in SQL in any way.
+  FIXED_UDA_INTERMEDIATE("FIXED_UDA_INTERMEDIATE", -1,
+      TPrimitiveType.FIXED_UDA_INTERMEDIATE);
 
   private final String description_;
   private final int slotSize_;  // size of tuple slot for this type
@@ -86,6 +92,7 @@ public enum PrimitiveType {
       case CHAR: return CHAR;
       case DECIMAL: return DECIMAL;
       case BINARY: return BINARY;
+      case FIXED_UDA_INTERMEDIATE: return FIXED_UDA_INTERMEDIATE;
     }
     return INVALID_TYPE;
   }