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 2016/10/18 16:43:44 UTC

[21/32] incubator-impala git commit: IMPALA-2789: More compact mem layout with null bits at the end.

IMPALA-2789: More compact mem layout with null bits at the end.

There are two motivations for this change:
1. Reduce memory consumption.
2. Pave the way for full memory layout compatibility between
   Impala and Kudu to eventually enable zero-copy scans. This
   patch is a only first step towards that goal.

New Memory Layout
Slots are placed in descending order by size with trailing bytes to
store null flags. Null flags are omitted for non-nullable slots. There
is no padding between tuples when stored back-to-back in a row batch.

Example: select bool_col, int_col, string_col, smallint_col
         from functional.alltypes
Slots:   string_col|int_col|smallint_col|bool_col|null_byte
Offsets: 0          16      20           22       23

The main change is to move the null indicators to the end of tuples.
The new memory layout is fully packed with no padding in between
slots or tuples.

Performance:
Our standard cluster perf tests showed no significant difference in
query response times as well as consumed cycles, and a slight
reduction in peak memory consumption.

Testing:
An exhaustive test run passed. Ran a few select tests like TPC-H/DS
with ASAN locally.

These follow-on changes are planned:
1. Planner needs to mark slots non-nullable if they correspond
   to a non-nullable Kudu column.
2. Update Kudu scan node to copy tuples with memcpy.
3. Kudu client needs to support transferring ownership of the
   tuple memory (maybe do direct and indirect buffers separately).
4. Update Kudu scan node to use memory transfer instead of copy

Change-Id: Ib6510c75d841bddafa6638f1bd2ac6731a7053f6
Reviewed-on: http://gerrit.cloudera.org:8080/4673
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal 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/b0e87c68
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/b0e87c68
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/b0e87c68

Branch: refs/heads/hadoop-next
Commit: b0e87c685d96e4e55a0fda3b23c10fc069f7551a
Parents: 9f61397
Author: Alex Behm <al...@cloudera.com>
Authored: Tue Dec 22 13:56:32 2015 -0800
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sun Oct 16 23:36:10 2016 +0000

----------------------------------------------------------------------
 .../benchmarks/row-batch-serialize-benchmark.cc |  13 ++-
 be/src/codegen/llvm-codegen.cc                  |  11 ++
 be/src/codegen/llvm-codegen.h                   |   5 +
 be/src/exec/hdfs-scanner.cc                     |  73 +++++++------
 be/src/exec/hdfs-scanner.h                      |  13 +--
 be/src/exec/kudu-scanner.cc                     |   7 +-
 be/src/exec/kudu-scanner.h                      |   3 -
 be/src/exec/row-batch-list-test.cc              |  20 +++-
 be/src/exec/text-converter.cc                   |   2 +-
 be/src/runtime/buffered-tuple-stream-test.cc    |  35 +++----
 be/src/runtime/collection-value-builder-test.cc |  15 ++-
 be/src/runtime/descriptors.cc                   |  56 +++++-----
 be/src/runtime/descriptors.h                    |   4 +-
 be/src/runtime/row-batch-serialize-test.cc      |  32 ++++--
 be/src/runtime/row-batch-test.cc                |  16 ++-
 be/src/runtime/tuple.cc                         |   6 +-
 be/src/runtime/tuple.h                          |   7 +-
 be/src/service/frontend.cc                      |   7 ++
 be/src/service/frontend.h                       |   8 ++
 be/src/testutil/desc-tbl-builder.cc             |  96 ++++-------------
 be/src/testutil/desc-tbl-builder.h              |  20 ++--
 common/thrift/Frontend.thrift                   |   7 ++
 .../apache/impala/analysis/DescriptorTable.java |  72 +++++++++++--
 .../apache/impala/analysis/TupleDescriptor.java |  43 ++++----
 .../org/apache/impala/service/JniFrontend.java  |  41 ++++++--
 .../apache/impala/analysis/AnalyzerTest.java    | 102 ++++++++-----------
 26 files changed, 402 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/benchmarks/row-batch-serialize-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/row-batch-serialize-benchmark.cc b/be/src/benchmarks/row-batch-serialize-benchmark.cc
index 3f04906..5a8a104 100644
--- a/be/src/benchmarks/row-batch-serialize-benchmark.cc
+++ b/be/src/benchmarks/row-batch-serialize-benchmark.cc
@@ -17,11 +17,15 @@
 
 #include <iostream>
 #include <sstream>
+#include <boost/scoped_ptr.hpp>
 
+#include "common/init.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/raw-value.h"
 #include "runtime/row-batch.h"
 #include "runtime/tuple-row.h"
+#include "service/fe-support.h"
+#include "service/frontend.h"
 #include "testutil/desc-tbl-builder.h"
 #include "util/benchmark.h"
 #include "util/compress.h"
@@ -88,6 +92,10 @@ const int NUM_ROWS = 1024;
 const int MAX_STRING_LEN = 10;
 
 namespace impala {
+
+// For computing tuple mem layouts.
+static scoped_ptr<Frontend> fe;
+
 // Friend class with access to RowBatch internals
 class RowBatchSerializeBaseline {
  public:
@@ -318,7 +326,7 @@ class RowBatchSerializeBenchmark {
     MemTracker tracker;
     MemPool mem_pool(&tracker);
     ObjectPool obj_pool;
-    DescriptorTblBuilder builder(&obj_pool);
+    DescriptorTblBuilder builder(fe.get(), &obj_pool);
     builder.DeclareTuple() << TYPE_INT << TYPE_STRING;
     DescriptorTbl* desc_tbl = builder.Build();
 
@@ -398,6 +406,9 @@ class RowBatchSerializeBenchmark {
 }
 
 int main(int argc, char** argv) {
+  impala::InitCommonRuntime(argc, argv, true);
+  InitFeSupport();
+  fe.reset(new Frontend());
   RowBatchSerializeBenchmark::Run();
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index cc8b46c..d43ad6e 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -59,6 +59,7 @@
 #include "codegen/instruction-counter.h"
 #include "codegen/mcjit-mem-mgr.h"
 #include "impala-ir/impala-ir-names.h"
+#include "runtime/descriptors.h"
 #include "runtime/hdfs-fs-cache.h"
 #include "runtime/lib-cache.h"
 #include "runtime/mem-pool.h"
@@ -1210,6 +1211,16 @@ void LlvmCodeGen::CodegenMemset(LlvmBuilder* builder, Value* dst, int value, int
   builder->CreateMemSet(dst, value_const, size, /* no alignment */ 0);
 }
 
+void LlvmCodeGen::CodegenClearNullBits(LlvmBuilder* builder, Value* tuple_ptr,
+    const TupleDescriptor& tuple_desc) {
+  Value* int8_ptr = builder->CreateBitCast(tuple_ptr, ptr_type(), "int8_ptr");
+  Value* null_bytes_offset =
+      ConstantInt::get(int_type(), tuple_desc.null_bytes_offset());
+  Value* null_bytes_ptr =
+      builder->CreateInBoundsGEP(int8_ptr, null_bytes_offset, "null_bytes_ptr");
+  CodegenMemset(builder, null_bytes_ptr, 0, tuple_desc.num_null_bytes());
+}
+
 Value* LlvmCodeGen::CodegenAllocate(LlvmBuilder* builder, MemPool* pool, Value* size,
     const char* name) {
   DCHECK(pool != NULL);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 2ef936f..0b81701 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -74,6 +74,7 @@ namespace impala {
 class CodegenAnyVal;
 class CodegenSymbolEmitter;
 class SubExprElimination;
+class TupleDescriptor;
 
 /// LLVM code generator.  This is the top level object to generate jitted code.
 //
@@ -419,6 +420,10 @@ class LlvmCodeGen {
   /// be a pointer. No-op if size is zero.
   void CodegenMemset(LlvmBuilder* builder, llvm::Value* dst, int value, int size);
 
+  /// Codegen to set all null bytes of the given tuple to 0.
+  void CodegenClearNullBits(LlvmBuilder* builder, llvm::Value* tuple_ptr,
+      const TupleDescriptor& tuple_desc);
+
   /// Codegen to call pool->Allocate(size).
   llvm::Value* CodegenAllocate(LlvmBuilder* builder, MemPool* pool, llvm::Value* size,
       const char* name = "");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index 28eb606..81542ec 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -67,7 +67,6 @@ HdfsScanner::HdfsScanner(HdfsScanNodeBase* scan_node, RuntimeState* state)
       template_tuple_pool_(new MemPool(scan_node->mem_tracker())),
       template_tuple_(NULL),
       tuple_byte_size_(scan_node->tuple_desc()->byte_size()),
-      num_null_bytes_(scan_node->tuple_desc()->num_null_bytes()),
       tuple_(NULL),
       batch_(NULL),
       tuple_mem_(NULL),
@@ -88,7 +87,6 @@ HdfsScanner::HdfsScanner()
       template_tuple_pool_(NULL),
       template_tuple_(NULL),
       tuple_byte_size_(-1),
-      num_null_bytes_(-1),
       tuple_(NULL),
       batch_(NULL),
       tuple_mem_(NULL),
@@ -302,50 +300,54 @@ bool HdfsScanner::WriteCompleteTuple(MemPool* pool, FieldLocation* fields,
   return EvalConjuncts(tuple_row);
 }
 
-// Codegen for WriteTuple(above).  The signature matches WriteTuple (except for the
-// this* first argument).  For writing out and evaluating a single string slot:
+// Codegen for WriteTuple(above) for writing out single nullable string slot and
+// evaluating a <slot> = <constantexpr> conjunct. The signature matches WriteTuple()
+// except for the first this* argument.
 // define i1 @WriteCompleteTuple(%"class.impala::HdfsScanner"* %this,
 //                               %"class.impala::MemPool"* %pool,
 //                               %"struct.impala::FieldLocation"* %fields,
 //                               %"class.impala::Tuple"* %tuple,
 //                               %"class.impala::TupleRow"* %tuple_row,
 //                               %"class.impala::Tuple"* %template,
-//                               i8* %error_fields, i8* %error_in_row) #20 {
+//                               i8* %error_fields, i8* %error_in_row) {
 // entry:
 //   %tuple_ptr = bitcast %"class.impala::Tuple"* %tuple
-//                to { i8, %"struct.impala::StringValue" }*
+//                to <{ %"struct.impala::StringValue", i8 }>*
 //   %tuple_ptr1 = bitcast %"class.impala::Tuple"* %template
-//                 to { i8, %"struct.impala::StringValue" }*
-//   %null_byte = getelementptr inbounds
-//                { i8, %"struct.impala::StringValue" }* %tuple_ptr, i32 0, i32 0
-//   store i8 0, i8* %null_byte
+//                 to <{ %"struct.impala::StringValue", i8 }>*
+//   %int8_ptr = bitcast <{ %"struct.impala::StringValue", i8 }>* %tuple_ptr to i8*
+//   %null_bytes_ptr = getelementptr i8, i8* %int8_ptr, i32 16
+//   call void @llvm.memset.p0i8.i64(i8* %null_bytes_ptr, i8 0, i64 1, i32 0, i1 false)
 //   %0 = bitcast %"class.impala::TupleRow"* %tuple_row
-//        to { i8, %"struct.impala::StringValue" }**
-//   %1 = getelementptr { i8, %"struct.impala::StringValue" }** %0, i32 0
-//   store { i8, %"struct.impala::StringValue" }* %tuple_ptr,
-//         { i8, %"struct.impala::StringValue" }** %1
+//        to <{ %"struct.impala::StringValue", i8 }>**
+//   %1 = getelementptr <{ %"struct.impala::StringValue", i8 }>*,
+//                      <{ %"struct.impala::StringValue", i8 }>** %0, i32 0
+//   store <{ %"struct.impala::StringValue", i8 }>* %tuple_ptr,
+//         <{ %"struct.impala::StringValue", i8 }>** %1
 //   br label %parse
 //
 // parse:                                            ; preds = %entry
-//   %data_ptr = getelementptr %"struct.impala::FieldLocation"* %fields, i32 0, i32 0
-//   %len_ptr = getelementptr %"struct.impala::FieldLocation"* %fields, i32 0, i32 1
-//   %slot_error_ptr = getelementptr i8* %error_fields, i32 0
-//   %data = load i8** %data_ptr
-//   %len = load i32* %len_ptr
-//   %2 = call i1 @WriteSlot({ i8, %"struct.impala::StringValue" }* %tuple_ptr,
-//                           i8* %data, i32 %len)
-//   %slot_parse_error = xor i1 %2, true
-//   %error_in_row2 = or i1 false, %slot_parse_error
-//   %3 = zext i1 %slot_parse_error to i8
-//   store i8 %3, i8* %slot_error_ptr
-//   %4 = call %"class.impala::ExprContext"* @GetConjunctCtx(
-//       %"class.impala::HdfsScanner"* %this, i32 0)
-//   %conjunct_eval = call i16 @Eq_StringVal_StringValWrapper1(
-//       %"class.impala::ExprContext"* %4, %"class.impala::TupleRow"* %tuple_row)
-//   %5 = ashr i16 %conjunct_eval, 8
-//   %6 = trunc i16 %5 to i8
-//   %val = trunc i8 %6 to i1
-//   br i1 %val, label %parse3, label %eval_fail
+//  %data_ptr = getelementptr %"struct.impala::FieldLocation",
+//                            %"struct.impala::FieldLocation"* %fields, i32 0, i32 0
+//  %len_ptr = getelementptr %"struct.impala::FieldLocation",
+//                           %"struct.impala::FieldLocation"* %fields, i32 0, i32 1
+//  %slot_error_ptr = getelementptr i8, i8* %error_fields, i32 0
+//  %data = load i8*, i8** %data_ptr
+//  %len = load i32, i32* %len_ptr
+//  %2 = call i1 @WriteSlot(<{ %"struct.impala::StringValue", i8 }>* %tuple_ptr,
+//                          i8* %data, i32 %len)
+//  %slot_parse_error = xor i1 %2, true
+//  %error_in_row2 = or i1 false, %slot_parse_error
+//  %3 = zext i1 %slot_parse_error to i8
+//  store i8 %3, i8* %slot_error_ptr
+//  %4 = call %"class.impala::ExprContext"* @GetConjunctCtx(
+//    %"class.impala::HdfsScanner"* %this, i32 0)
+//  %conjunct_eval = call i16 @"impala::Operators::Eq_StringVal_StringValWrapper"(
+//    %"class.impala::ExprContext"* %4, %"class.impala::TupleRow"* %tuple_row)
+//  %5 = ashr i16 %conjunct_eval, 8
+//  %6 = trunc i16 %5 to i8
+//  %val = trunc i8 %6 to i1
+//  br i1 %val, label %parse3, label %eval_fail
 //
 // parse3:                                           ; preds = %parse
 //   %7 = zext i1 %error_in_row2 to i8
@@ -451,10 +453,7 @@ Status HdfsScanner::CodegenWriteCompleteTuple(HdfsScanNodeBase* node,
   // Initialize tuple
   if (node->num_materialized_partition_keys() == 0) {
     // No partition key slots, just zero the NULL bytes.
-    for (int i = 0; i < tuple_desc->num_null_bytes(); ++i) {
-      Value* null_byte = builder.CreateStructGEP(NULL, tuple_arg, i, "null_byte");
-      builder.CreateStore(codegen->GetIntConstant(TYPE_TINYINT, 0), null_byte);
-    }
+    codegen->CodegenClearNullBits(&builder, tuple_arg, *tuple_desc);
   } else {
     // Copy template tuple.
     // TODO: only copy what's necessary from the template tuple.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/hdfs-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.h b/be/src/exec/hdfs-scanner.h
index d9fa424..4a4d366 100644
--- a/be/src/exec/hdfs-scanner.h
+++ b/be/src/exec/hdfs-scanner.h
@@ -32,6 +32,7 @@
 #include "exec/scanner-context.h"
 #include "runtime/disk-io-mgr.h"
 #include "runtime/row-batch.h"
+#include "runtime/tuple.h"
 
 namespace impala {
 
@@ -44,7 +45,6 @@ class MemPool;
 class SlotDescriptor;
 class Status;
 class TextConverter;
-class Tuple;
 class TupleDescriptor;
 class TPlanNode;
 class TScanRange;
@@ -234,9 +234,6 @@ class HdfsScanner {
   /// Fixed size of each top-level tuple, in bytes
   const int32_t tuple_byte_size_;
 
-  /// Number of null bytes in the top-level tuple.
-  const int32_t num_null_bytes_;
-
   /// Current tuple pointer into tuple_mem_.
   Tuple* tuple_;
 
@@ -443,18 +440,14 @@ class HdfsScanner {
     if (template_tuple != NULL) {
       memcpy(tuple, template_tuple, desc->byte_size());
     } else {
-      memset(tuple, 0, sizeof(uint8_t) * desc->num_null_bytes());
+      tuple->ClearNullBits(*desc);
     }
   }
 
   // TODO: replace this function with above once we can inline constants from
   // scan_node_->tuple_desc() via codegen
   void InitTuple(Tuple* template_tuple, Tuple* tuple) {
-    if (template_tuple != NULL) {
-      memcpy(tuple, template_tuple, tuple_byte_size_);
-    } else {
-      memset(tuple, 0, sizeof(uint8_t) * num_null_bytes_);
-    }
+    InitTuple(scan_node_->tuple_desc(), template_tuple, tuple);
   }
 
   inline Tuple* next_tuple(int tuple_byte_size, Tuple* t) const {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index 4fcb40a..a6affbf 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -59,7 +59,6 @@ KuduScanner::KuduScanner(KuduScanNode* scan_node, RuntimeState* state)
     state_(state),
     cur_kudu_batch_num_read_(0),
     last_alive_time_micros_(0),
-    tuple_num_null_bytes_(scan_node_->tuple_desc()->num_null_bytes()),
     num_string_slots_(0) {
 }
 
@@ -185,14 +184,14 @@ Status KuduScanner::DecodeRowsIntoRowBatch(RowBatch* row_batch,
   // that happens inside the loop.
   int idx = row_batch->AddRow();
   TupleRow* row = row_batch->GetRow(idx);
-  (*tuple_mem)->Init(scan_node_->tuple_desc()->num_null_bytes());
+  (*tuple_mem)->ClearNullBits(*scan_node_->tuple_desc());
   row->SetTuple(tuple_idx(), *tuple_mem);
 
   int num_rows = cur_kudu_batch_.NumRows();
   // Now iterate through the Kudu rows.
   for (int krow_idx = cur_kudu_batch_num_read_; krow_idx < num_rows; ++krow_idx) {
     // Clear any NULL indicators set by a previous iteration.
-    (*tuple_mem)->Init(tuple_num_null_bytes_);
+    (*tuple_mem)->ClearNullBits(*scan_node_->tuple_desc());
 
     // Transform a Kudu row into an Impala row.
     KuduScanBatch::RowPtr krow = cur_kudu_batch_.Row(krow_idx);
@@ -216,7 +215,7 @@ Status KuduScanner::DecodeRowsIntoRowBatch(RowBatch* row_batch,
 
       // Move to the next tuple in the tuple buffer.
       *tuple_mem = next_tuple(*tuple_mem);
-      (*tuple_mem)->Init(tuple_num_null_bytes_);
+      (*tuple_mem)->ClearNullBits(*scan_node_->tuple_desc());
       // Make 'row' point to the new row.
       row = row_batch->GetRow(idx);
       row->SetTuple(tuple_idx(), *tuple_mem);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/kudu-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.h b/be/src/exec/kudu-scanner.h
index d868b05..0ed5221 100644
--- a/be/src/exec/kudu-scanner.h
+++ b/be/src/exec/kudu-scanner.h
@@ -121,9 +121,6 @@ class KuduScanner {
   /// The scanner's cloned copy of the conjuncts to apply.
   vector<ExprContext*> conjunct_ctxs_;
 
-  /// Number of bytes needed to represent the null bits in the tuple.
-  int tuple_num_null_bytes_;
-
   /// List of string slots that need relocation for their auxiliary memory.
   std::vector<SlotDescriptor*> string_slots_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/row-batch-list-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/row-batch-list-test.cc b/be/src/exec/row-batch-list-test.cc
index 38ce177..ad3317b 100644
--- a/be/src/exec/row-batch-list-test.cc
+++ b/be/src/exec/row-batch-list-test.cc
@@ -19,21 +19,30 @@
 #include <cstdio>
 #include <iostream>
 #include <vector>
+#include <boost/scoped_ptr.hpp>
 
+#include "common/init.h"
 #include "exec/row-batch-list.h"
 #include "runtime/descriptors.h"
 #include "runtime/mem-pool.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/string-value.h"
 #include "runtime/tuple-row.h"
+#include "service/fe-support.h"
+#include "service/frontend.h"
 #include "util/runtime-profile-counters.h"
 #include "testutil/desc-tbl-builder.h"
 #include "testutil/gtest-util.h"
 
 #include "common/names.h"
 
+using namespace impala;
+
 namespace impala {
 
+// For computing tuple mem layouts.
+scoped_ptr<Frontend> fe;
+
 class RowBatchListTest : public testing::Test {
  public:
   RowBatchListTest() {}
@@ -44,7 +53,7 @@ class RowBatchListTest : public testing::Test {
   RowDescriptor* desc_;
 
   virtual void SetUp() {
-    DescriptorTblBuilder builder(&pool_);
+    DescriptorTblBuilder builder(fe.get(), &pool_);
     builder.DeclareTuple() << TYPE_INT;
     DescriptorTbl* desc_tbl = builder.Build();
     vector<bool> nullable_tuples(1, false);
@@ -139,5 +148,10 @@ TEST_F(RowBatchListTest, MultipleRowBatchesTest) {
 
 }
 
-IMPALA_TEST_MAIN();
-
+int main(int argc, char** argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+  InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  InitFeSupport();
+  fe.reset(new Frontend());
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/exec/text-converter.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/text-converter.cc b/be/src/exec/text-converter.cc
index 2271b4e..d38d662 100644
--- a/be/src/exec/text-converter.cc
+++ b/be/src/exec/text-converter.cc
@@ -126,7 +126,7 @@ Function* TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
 
   StructType* tuple_type = tuple_desc->GetLlvmStruct(codegen);
   if (tuple_type == NULL) return NULL;
-  PointerType* tuple_ptr_type = PointerType::get(tuple_type, 0);
+  PointerType* tuple_ptr_type = tuple_type->getPointerTo();
 
   Function* set_null_fn = slot_desc->GetUpdateNullFn(codegen, true);
   if (set_null_fn == NULL) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/buffered-tuple-stream-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream-test.cc b/be/src/runtime/buffered-tuple-stream-test.cc
index 20af23e..76b1bff 100644
--- a/be/src/runtime/buffered-tuple-stream-test.cc
+++ b/be/src/runtime/buffered-tuple-stream-test.cc
@@ -79,12 +79,12 @@ class SimpleTupleStreamTest : public testing::Test {
     vector<bool> nullable_tuples(1, false);
     vector<TTupleId> tuple_ids(1, static_cast<TTupleId>(0));
 
-    DescriptorTblBuilder int_builder(&pool_);
+    DescriptorTblBuilder int_builder(test_env_->exec_env()->frontend(), &pool_);
     int_builder.DeclareTuple() << TYPE_INT;
     int_desc_ = pool_.Add(new RowDescriptor(
         *int_builder.Build(), tuple_ids, nullable_tuples));
 
-    DescriptorTblBuilder string_builder(&pool_);
+    DescriptorTblBuilder string_builder(test_env_->exec_env()->frontend(), &pool_);
     string_builder.DeclareTuple() << TYPE_STRING;
     string_desc_ = pool_.Add(new RowDescriptor(
         *string_builder.Build(), tuple_ids, nullable_tuples));
@@ -145,23 +145,21 @@ class SimpleTupleStreamTest : public testing::Test {
       for (int tuple_idx = 0; tuple_idx < num_tuples; ++tuple_idx) {
         TupleDescriptor* tuple_desc = row_desc.tuple_descriptors()[tuple_idx];
         Tuple* tuple = Tuple::Create(tuple_desc->byte_size(), batch->tuple_data_pool());
-        // Skip over the null indicators at the beginning of the tuple.
-        uint8_t* ptr = reinterpret_cast<uint8_t*>(tuple) + tuple_desc->num_null_bytes();
         bool is_null = gen_null && !GenBoolValue(idx);
         for (int slot_idx = 0; slot_idx < tuple_desc->slots().size(); ++slot_idx, ++idx) {
           SlotDescriptor* slot_desc = tuple_desc->slots()[slot_idx];
+          void* slot = tuple->GetSlot(slot_desc->tuple_offset());
           switch (slot_desc->type().type) {
             case TYPE_INT:
-              *reinterpret_cast<int*>(ptr) = GenIntValue(idx);
+              *reinterpret_cast<int*>(slot) = GenIntValue(idx);
               break;
             case TYPE_STRING:
-              *reinterpret_cast<StringValue*>(ptr) = STRINGS[idx % NUM_STRINGS];
+              *reinterpret_cast<StringValue*>(slot) = STRINGS[idx % NUM_STRINGS];
               break;
             default:
               // The memory has been zero'ed out already by Tuple::Create().
               break;
           }
-          ptr += slot_desc->slot_size();
         }
         if (is_null) {
           row->SetTuple(tuple_idx, NULL);
@@ -212,14 +210,13 @@ class SimpleTupleStreamTest : public testing::Test {
       TupleDescriptor* tuple_desc = row_desc->tuple_descriptors()[tuple_idx];
       Tuple* tuple = row->GetTuple(tuple_idx);
       const int num_slots = tuple_desc->slots().size();
-      uint8_t* ptr = reinterpret_cast<uint8_t*>(tuple) + tuple_desc->num_null_bytes();
       for (int slot_idx = 0; slot_idx < num_slots; ++slot_idx) {
         SlotDescriptor* slot_desc = tuple_desc->slots()[slot_idx];
         if (tuple == NULL) {
           AppendValue(NULL, results);
         } else {
-          AppendValue(ptr, results);
-          ptr += slot_desc->slot_size();
+          void* slot = tuple->GetSlot(slot_desc->tuple_offset());
+          AppendValue(reinterpret_cast<uint8_t*>(slot), results);
         }
       }
     }
@@ -270,7 +267,7 @@ class SimpleTupleStreamTest : public testing::Test {
         for (int slot_idx = 0; slot_idx < num_slots; ++slot_idx, ++idx) {
           T expected_val;
           GetExpectedValue(idx, is_null, &expected_val);
-          ASSERT_TRUE(results[idx] == expected_val)
+          ASSERT_EQ(results[idx], expected_val)
               << "results[" << idx << "] " << results[idx] << " != "
               << expected_val << " row_idx=" << row_idx
               << " tuple_idx=" << tuple_idx << " slot_idx=" << slot_idx
@@ -400,12 +397,12 @@ class SimpleNullStreamTest : public SimpleTupleStreamTest {
     vector<bool> nullable_tuples(1, true);
     vector<TTupleId> tuple_ids(1, static_cast<TTupleId>(0));
 
-    DescriptorTblBuilder int_builder(&pool_);
+    DescriptorTblBuilder int_builder(test_env_->exec_env()->frontend(), &pool_);
     int_builder.DeclareTuple() << TYPE_INT;
     int_desc_ = pool_.Add(new RowDescriptor(
         *int_builder.Build(), tuple_ids, nullable_tuples));
 
-    DescriptorTblBuilder string_builder(&pool_);
+    DescriptorTblBuilder string_builder(test_env_->exec_env()->frontend(), &pool_);
     string_builder.DeclareTuple() << TYPE_STRING;
     string_desc_ = pool_.Add(new RowDescriptor(
         *string_builder.Build(), tuple_ids, nullable_tuples));
@@ -426,14 +423,14 @@ class MultiTupleStreamTest : public SimpleTupleStreamTest {
     tuple_ids.push_back(static_cast<TTupleId>(1));
     tuple_ids.push_back(static_cast<TTupleId>(2));
 
-    DescriptorTblBuilder int_builder(&pool_);
+    DescriptorTblBuilder int_builder(test_env_->exec_env()->frontend(), &pool_);
     int_builder.DeclareTuple() << TYPE_INT;
     int_builder.DeclareTuple() << TYPE_INT;
     int_builder.DeclareTuple() << TYPE_INT;
     int_desc_ = pool_.Add(new RowDescriptor(
         *int_builder.Build(), tuple_ids, nullable_tuples));
 
-    DescriptorTblBuilder string_builder(&pool_);
+    DescriptorTblBuilder string_builder(test_env_->exec_env()->frontend(), &pool_);
     string_builder.DeclareTuple() << TYPE_STRING;
     string_builder.DeclareTuple() << TYPE_STRING;
     string_builder.DeclareTuple() << TYPE_STRING;
@@ -456,14 +453,14 @@ class MultiNullableTupleStreamTest : public SimpleTupleStreamTest {
     tuple_ids.push_back(static_cast<TTupleId>(1));
     tuple_ids.push_back(static_cast<TTupleId>(2));
 
-    DescriptorTblBuilder int_builder(&pool_);
+    DescriptorTblBuilder int_builder(test_env_->exec_env()->frontend(), &pool_);
     int_builder.DeclareTuple() << TYPE_INT;
     int_builder.DeclareTuple() << TYPE_INT;
     int_builder.DeclareTuple() << TYPE_INT;
     int_desc_ = pool_.Add(new RowDescriptor(
         *int_builder.Build(), tuple_ids, nullable_tuples));
 
-    DescriptorTblBuilder string_builder(&pool_);
+    DescriptorTblBuilder string_builder(test_env_->exec_env()->frontend(), &pool_);
     string_builder.DeclareTuple() << TYPE_STRING;
     string_builder.DeclareTuple() << TYPE_STRING;
     string_builder.DeclareTuple() << TYPE_STRING;
@@ -495,7 +492,7 @@ class ArrayTupleStreamTest : public SimpleTupleStreamTest {
     nested_array_type.type = TYPE_ARRAY;
     nested_array_type.children.push_back(int_array_type);
 
-    DescriptorTblBuilder builder(&pool_);
+    DescriptorTblBuilder builder(test_env_->exec_env()->frontend(), &pool_);
     builder.DeclareTuple() << string_array_type << nested_array_type;
     builder.DeclareTuple() << int_array_type;
     array_desc_ = pool_.Add(new RowDescriptor(
@@ -735,7 +732,7 @@ TEST_F(SimpleTupleStreamTest, BigRow) {
   vector<bool> nullable_tuples;
   vector<bool> non_nullable_tuples;
 
-  DescriptorTblBuilder big_row_builder(&pool_);
+  DescriptorTblBuilder big_row_builder(test_env_->exec_env()->frontend(), &pool_);
   // Each tuple contains 8 slots of TYPE_INT and a single byte for null indicator.
   const int num_tuples = IO_BLOCK_SIZE / (8 * sizeof(int) + 1);
   for (int tuple_idx = 0; tuple_idx < num_tuples; ++tuple_idx) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/collection-value-builder-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/collection-value-builder-test.cc b/be/src/runtime/collection-value-builder-test.cc
index b8f4b65..613132a 100644
--- a/be/src/runtime/collection-value-builder-test.cc
+++ b/be/src/runtime/collection-value-builder-test.cc
@@ -17,6 +17,8 @@
 
 #include "runtime/collection-value-builder.h"
 #include "runtime/mem-tracker.h"
+#include "service/fe-support.h"
+#include "service/frontend.h"
 #include "testutil/desc-tbl-builder.h"
 #include "testutil/gtest-util.h"
 
@@ -24,9 +26,12 @@
 
 using namespace impala;
 
+// For computing tuple mem layouts.
+static scoped_ptr<Frontend> fe;
+
 TEST(CollectionValueBuilderTest, MaxBufferSize) {
   ObjectPool obj_pool;
-  DescriptorTblBuilder builder(&obj_pool);
+  DescriptorTblBuilder builder(fe.get(), &obj_pool);
   builder.DeclareTuple() << TYPE_TINYINT << TYPE_TINYINT << TYPE_TINYINT;
   DescriptorTbl* desc_tbl = builder.Build();
   vector<TupleDescriptor*> descs;
@@ -66,4 +71,10 @@ TEST(CollectionValueBuilderTest, MaxBufferSize) {
   pool.FreeAll();
 }
 
-IMPALA_TEST_MAIN();
+int main(int argc, char** argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+  InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  InitFeSupport();
+  fe.reset(new Frontend());
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/descriptors.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/descriptors.cc b/be/src/runtime/descriptors.cc
index 47ca791..d106ed4 100644
--- a/be/src/runtime/descriptors.cc
+++ b/be/src/runtime/descriptors.cc
@@ -286,6 +286,7 @@ TupleDescriptor::TupleDescriptor(const TTupleDescriptor& tdesc)
     table_desc_(NULL),
     byte_size_(tdesc.byteSize),
     num_null_bytes_(tdesc.numNullBytes),
+    null_bytes_offset_(tdesc.byteSize - tdesc.numNullBytes),
     slots_(),
     has_varlen_slots_(false),
     tuple_path_(tdesc.tuplePath),
@@ -591,14 +592,18 @@ Function* SlotDescriptor::GetUpdateNullFn(LlvmCodeGen* codegen, bool set_null) c
   prototype.AddArgument(LlvmCodeGen::NamedVariable("tuple", tuple_ptr_type));
 
   LlvmCodeGen::LlvmBuilder builder(codegen->context());
-  Value* tuple_ptr;
-  Function* fn = prototype.GeneratePrototype(&builder, &tuple_ptr);
-
-  Value* null_byte_ptr = builder.CreateStructGEP(NULL,
-          tuple_ptr, null_indicator_offset_.byte_offset, "null_byte_ptr");
+  Value* tuple_arg;
+  Function* fn = prototype.GeneratePrototype(&builder, &tuple_arg);
+
+  Value* tuple_int8_ptr =
+      builder.CreateBitCast(tuple_arg, codegen->ptr_type(), "tuple_int8_ptr");
+  Value* null_byte_offset =
+      ConstantInt::get(codegen->int_type(), null_indicator_offset_.byte_offset);
+  Value* null_byte_ptr =
+      builder.CreateInBoundsGEP(tuple_int8_ptr, null_byte_offset, "null_byte_ptr");
   Value* null_byte = builder.CreateLoad(null_byte_ptr, "null_byte");
-  Value* result = NULL;
 
+  Value* result = NULL;
   if (set_null) {
     Value* null_set = codegen->GetIntConstant(
         TYPE_TINYINT, null_indicator_offset_.bit_mask);
@@ -627,45 +632,38 @@ StructType* TupleDescriptor::GetLlvmStruct(LlvmCodeGen* codegen) const {
 
   // Sort slots in the order they will appear in LLVM struct.
   vector<SlotDescriptor*> sorted_slots(slots_.size());
-  for (SlotDescriptor* slot: slots_) {
-    sorted_slots[slot->slot_idx_] = slot;
-  }
-
-  // For each null byte, add a byte to the struct
-  vector<Type*> struct_fields;
-  for (int i = 0; i < num_null_bytes_; ++i) {
-    struct_fields.push_back(codegen->GetType(TYPE_TINYINT));
-  }
-  int curr_struct_offset = num_null_bytes_;
+  for (SlotDescriptor* slot: slots_) sorted_slots[slot->slot_idx_] = slot;
 
   // Add the slot types to the struct description.
+  vector<Type*> struct_fields;
+  int curr_struct_offset = 0;
   for (SlotDescriptor* slot: sorted_slots) {
     // IMPALA-3207: Codegen for CHAR is not yet implemented: bail out of codegen here.
     if (slot->type().type == TYPE_CHAR) return NULL;
-    DCHECK_LE(curr_struct_offset, slot->tuple_offset());
-    if (curr_struct_offset < slot->tuple_offset()) {
-      // Need to add padding to ensure slots are aligned correctly. Clang likes to
-      // sometimes pad structs in its own way. When it does this, it sets the 'packed'
-      // flag, which means that at the LLVM level the struct type has no alignment
-      // requirements, even if it does at the C language level.
-      struct_fields.push_back(ArrayType::get(codegen->GetType(TYPE_TINYINT),
-          slot->tuple_offset() - curr_struct_offset));
-    }
+    DCHECK_EQ(curr_struct_offset, slot->tuple_offset());
     slot->llvm_field_idx_ = struct_fields.size();
     struct_fields.push_back(codegen->GetType(slot->type()));
     curr_struct_offset = slot->tuple_offset() + slot->slot_size();
   }
+  // For each null byte, add a byte to the struct
+  for (int i = 0; i < num_null_bytes_; ++i) {
+    struct_fields.push_back(codegen->GetType(TYPE_TINYINT));
+    ++curr_struct_offset;
+  }
+
   DCHECK_LE(curr_struct_offset, byte_size_);
   if (curr_struct_offset < byte_size_) {
     struct_fields.push_back(ArrayType::get(codegen->GetType(TYPE_TINYINT),
         byte_size_ - curr_struct_offset));
   }
 
-  // Construct the struct type.
-  // We don't mark the struct as packed but it shouldn't matter either way: LLVM should
-  // not insert any additional padding since the contents are already aligned.
+  // Construct the struct type. Use the packed layout although not strictly necessary
+  // because the fields are already aligned, so LLVM should not add any padding. The
+  // fields are already aligned because we order the slots by descending size and only
+  // have powers-of-two slot sizes. Note that STRING and TIMESTAMP slots both occupy
+  // 16 bytes although their useful payload is only 12 bytes.
   StructType* tuple_struct = StructType::get(codegen->context(),
-      ArrayRef<Type*>(struct_fields));
+      ArrayRef<Type*>(struct_fields), true);
   const DataLayout& data_layout = codegen->execution_engine()->getDataLayout();
   const StructLayout* layout = data_layout.getStructLayout(tuple_struct);
   for (SlotDescriptor* slot: slots()) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/descriptors.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h
index 874ad69..02193df 100644
--- a/be/src/runtime/descriptors.h
+++ b/be/src/runtime/descriptors.h
@@ -160,7 +160,7 @@ class SlotDescriptor {
 
   /// The idx of the slot in the llvm codegen'd tuple struct
   /// This is set by TupleDescriptor during codegen and takes into account
-  /// leading null bytes and any padding bytes.
+  /// any padding bytes.
   int llvm_field_idx_;
 
   /// Cached codegen'd functions
@@ -363,6 +363,7 @@ class TupleDescriptor {
  public:
   int byte_size() const { return byte_size_; }
   int num_null_bytes() const { return num_null_bytes_; }
+  int null_bytes_offset() const { return null_bytes_offset_; }
   const std::vector<SlotDescriptor*>& slots() const { return slots_; }
   const std::vector<SlotDescriptor*>& string_slots() const { return string_slots_; }
   const std::vector<SlotDescriptor*>& collection_slots() const {
@@ -401,6 +402,7 @@ class TupleDescriptor {
   TableDescriptor* table_desc_;
   const int byte_size_;
   const int num_null_bytes_;
+  const int null_bytes_offset_;
 
   /// Contains all slots.
   std::vector<SlotDescriptor*> slots_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/row-batch-serialize-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch-serialize-test.cc b/be/src/runtime/row-batch-serialize-test.cc
index 0f5e519..f4c4f0b 100644
--- a/be/src/runtime/row-batch-serialize-test.cc
+++ b/be/src/runtime/row-batch-serialize-test.cc
@@ -24,11 +24,15 @@
 #include "runtime/raw-value.inline.h"
 #include "runtime/row-batch.h"
 #include "runtime/tuple-row.h"
+#include "service/fe-support.h"
+#include "service/frontend.h"
 #include "util/stopwatch.h"
 #include "testutil/desc-tbl-builder.h"
 
 #include "common/names.h"
 
+using namespace impala;
+
 namespace impala {
 
 const int NUM_ROWS = 20;
@@ -41,13 +45,18 @@ class RowBatchSerializeTest : public testing::Test {
   ObjectPool pool_;
   scoped_ptr<MemTracker> tracker_;
 
+  // For computing tuple mem layouts.
+  scoped_ptr<Frontend> fe_;
+
   virtual void SetUp() {
+    fe_.reset(new Frontend());
     tracker_.reset(new MemTracker());
   }
 
   virtual void TearDown() {
     pool_.Clear();
     tracker_.reset();
+    fe_.reset();
   }
 
   // Serializes and deserializes 'batch', then checks that the deserialized batch is valid
@@ -291,7 +300,7 @@ class RowBatchSerializeTest : public testing::Test {
 
 TEST_F(RowBatchSerializeTest, Basic) {
   // tuple: (int)
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -306,7 +315,7 @@ TEST_F(RowBatchSerializeTest, Basic) {
 
 TEST_F(RowBatchSerializeTest, String) {
   // tuple: (int, string)
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT << TYPE_STRING;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -325,7 +334,7 @@ TEST_F(RowBatchSerializeTest, BasicArray) {
   array_type.type = TYPE_ARRAY;
   array_type.children.push_back(TYPE_INT);
 
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT << TYPE_STRING << array_type;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -353,7 +362,7 @@ TEST_F(RowBatchSerializeTest, StringArray) {
   array_type.type = TYPE_ARRAY;
   array_type.children.push_back(struct_type);
 
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT << TYPE_STRING << array_type;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -394,7 +403,7 @@ TEST_F(RowBatchSerializeTest, NestedArrays) {
   array_type.type = TYPE_ARRAY;
   array_type.children.push_back(struct_type);
 
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << array_type;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -418,7 +427,7 @@ TEST_F(RowBatchSerializeTest, DupCorrectnessFull) {
 
 void RowBatchSerializeTest::TestDupCorrectness(bool full_dedup) {
   // tuples: (int), (string)
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT;
   builder.DeclareTuple() << TYPE_STRING;
   DescriptorTbl* desc_tbl = builder.Build();
@@ -459,7 +468,7 @@ TEST_F(RowBatchSerializeTest, DupRemovalFull) {
 // Test that tuple deduplication results in the expected reduction in serialized size.
 void RowBatchSerializeTest::TestDupRemoval(bool full_dedup) {
   // tuples: (int, string)
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT << TYPE_STRING;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -498,7 +507,7 @@ TEST_F(RowBatchSerializeTest, ConsecutiveNullsFull) {
 // Test that deduplication handles NULL tuples correctly.
 void RowBatchSerializeTest::TestConsecutiveNulls(bool full_dedup) {
   // tuples: (int)
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT;
   DescriptorTbl* desc_tbl = builder.Build();
   vector<bool> nullable_tuples(1, true);
@@ -526,7 +535,7 @@ TEST_F(RowBatchSerializeTest, ZeroLengthTuplesDedup) {
 
 void RowBatchSerializeTest::TestZeroLengthTuple(bool full_dedup) {
   // tuples: (int), (string), ()
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT;
   builder.DeclareTuple() << TYPE_STRING;
   builder.DeclareTuple();
@@ -553,7 +562,7 @@ TEST_F(RowBatchSerializeTest, DedupPathologicalFull) {
   ColumnType array_type;
   array_type.type = TYPE_ARRAY;
   array_type.children.push_back(TYPE_STRING);
-  DescriptorTblBuilder builder(&pool_);
+  DescriptorTblBuilder builder(fe_.get(), &pool_);
   builder.DeclareTuple() << TYPE_INT;
   builder.DeclareTuple() << TYPE_INT;
   builder.DeclareTuple() << array_type;
@@ -647,7 +656,8 @@ TEST_F(RowBatchSerializeTest, DedupPathologicalFull) {
 
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
-  impala::InitCommonRuntime(argc, argv, false, impala::TestInfo::BE_TEST);
+  InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  InitFeSupport();
   uint32_t seed = time(NULL);
   cout << "seed = " << seed << endl;
   srand(seed);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/row-batch-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch-test.cc b/be/src/runtime/row-batch-test.cc
index 2a8304a..041c3c2 100644
--- a/be/src/runtime/row-batch-test.cc
+++ b/be/src/runtime/row-batch-test.cc
@@ -15,23 +15,30 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <boost/scoped_ptr.hpp>
+
 #include "testutil/death-test-util.h"
 #include "testutil/gtest-util.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
+#include "service/fe-support.h"
+#include "service/frontend.h"
 #include "testutil/desc-tbl-builder.h"
 
 #include <gtest/gtest.h>
 
 #include "common/names.h"
 
-namespace impala {
+using namespace impala;
+
+// For computing tuple mem layouts.
+static scoped_ptr<Frontend> fe;
 
 TEST(RowBatchTest, AcquireStateWithMarkAtCapacity) {
   // Test that AcquireState() can be correctly called with MarkAtCapacity() on the
   // source batch.
   ObjectPool pool;
-  DescriptorTblBuilder builder(&pool);
+  DescriptorTblBuilder builder(fe.get(), &pool);
   builder.DeclareTuple() << TYPE_INT;
   DescriptorTbl* desc_tbl = builder.Build();
 
@@ -61,9 +68,10 @@ TEST(RowBatchTest, AcquireStateWithMarkAtCapacity) {
   }
 }
 
-}
-
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
+  InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  InitFeSupport();
+  fe.reset(new Frontend());
   return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/tuple.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tuple.cc b/be/src/runtime/tuple.cc
index df0066f..656621d 100644
--- a/be/src/runtime/tuple.cc
+++ b/be/src/runtime/tuple.cc
@@ -207,7 +207,7 @@ void Tuple::MaterializeExprs(
     TupleRow* row, const TupleDescriptor& desc, ExprContext* const* materialize_expr_ctxs,
     MemPool* pool, StringValue** non_null_string_values, int* total_string_lengths,
     int* num_non_null_string_values) {
-  memset(this, 0, desc.num_null_bytes());
+  ClearNullBits(desc);
   // Evaluate the materialize_expr_ctxs and place the results in the tuple.
   for (int i = 0; i < desc.slots().size(); ++i) {
     SlotDescriptor* slot_desc = desc.slots()[i];
@@ -368,8 +368,8 @@ Status Tuple::CodegenMaterializeExprs(RuntimeState* state, bool collect_string_v
   PointerType* tuple_type = codegen->GetPtrType(tuple_struct_type);
   Value* tuple = builder.CreateBitCast(opaque_tuple_arg, tuple_type, "tuple");
 
-  // Memset tuple's null bytes
-  codegen->CodegenMemset(&builder, tuple, 0, desc.num_null_bytes());
+  // Clear tuple's null bytes
+  codegen->CodegenClearNullBits(&builder, tuple, desc);
 
   // Evaluate the materialize_expr_ctxs and place the results in the tuple.
   for (int i = 0; i < desc.slots().size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/runtime/tuple.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/tuple.h b/be/src/runtime/tuple.h
index 3a3e399..b95492c 100644
--- a/be/src/runtime/tuple.h
+++ b/be/src/runtime/tuple.h
@@ -71,6 +71,11 @@ class Tuple {
 
   void Init(int size) { memset(this, 0, size); }
 
+  void ClearNullBits(const TupleDescriptor& tuple_desc) {
+    memset(reinterpret_cast<uint8_t*>(this) + tuple_desc.null_bytes_offset(),
+        0, tuple_desc.num_null_bytes());
+  }
+
   /// The total size of all data represented in this tuple (tuple data and referenced
   /// string and collection data).
   int64_t TotalByteSize(const TupleDescriptor& desc) const;
@@ -167,7 +172,7 @@ class Tuple {
       MemPool* pool, llvm::Function** fn);
 
   /// Turn null indicator bit on. For non-nullable slots, the mask will be 0 and
-  /// this is a no-op (but we don't have to branch to check is slots are nulalble).
+  /// this is a no-op (but we don't have to branch to check is slots are nullable).
   void SetNull(const NullIndicatorOffset& offset) {
     char* null_indicator_byte = reinterpret_cast<char*>(this) + offset.byte_offset;
     *null_indicator_byte |= offset.bit_mask;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/service/frontend.cc
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index 107354b..855924f 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -58,6 +58,7 @@ DEFINE_string(authorized_proxy_user_config, "",
     "users. For example: hue=user1,user2;admin=*");
 DEFINE_string(authorized_proxy_user_config_delimiter, ",",
     "Specifies the delimiter used in authorized_proxy_user_config. ");
+
 Frontend::Frontend() {
   JniMethodDescriptor methods[] = {
     {"<init>", "(ZLjava/lang/String;Ljava/lang/String;Ljava/lang/String;"
@@ -85,6 +86,7 @@ Frontend::Frontend() {
     {"loadTableData", "([B)[B", &load_table_data_id_},
     {"getTableFiles", "([B)[B", &get_table_files_id_},
     {"showCreateFunction", "([B)Ljava/lang/String;", &show_create_function_id_},
+    {"buildTestDescriptorTable", "([B)[B", &build_test_descriptor_table_id_},
 };
 
   JNIEnv* jni_env = getJNIEnv();
@@ -264,3 +266,8 @@ Status Frontend::SetCatalogInitialized() {
 Status Frontend::GetTableFiles(const TShowFilesParams& params, TResultSet* result) {
   return JniUtil::CallJniMethod(fe_, get_table_files_id_, params, result);
 }
+
+Status Frontend::BuildTestDescriptorTable(const TBuildTestDescriptorTableParams& params,
+    TDescriptorTable* result) {
+  return JniUtil::CallJniMethod(fe_, build_test_descriptor_table_id_, params, result);
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/service/frontend.h
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.h b/be/src/service/frontend.h
index 29dec68..c5c4895 100644
--- a/be/src/service/frontend.h
+++ b/be/src/service/frontend.h
@@ -168,6 +168,10 @@ class Frontend {
   /// Call FE to get files info for a table or partition.
   Status GetTableFiles(const TShowFilesParams& params, TResultSet* result);
 
+  /// Creates a thrift descriptor table for testing.
+  Status BuildTestDescriptorTable(const TBuildTestDescriptorTableParams& params,
+      TDescriptorTable* result);
+
  private:
   /// Descriptor of Java Frontend class itself, used to create a new instance.
   jclass fe_class_;
@@ -196,6 +200,10 @@ class Frontend {
   jmethodID set_catalog_initialized_id_; // JniFrontend.setCatalogInitialized
   jmethodID get_table_files_id_; // JniFrontend.getTableFiles
   jmethodID show_create_function_id_; // JniFrontend.showCreateFunction
+
+  // Only used for testing.
+  jmethodID build_test_descriptor_table_id_; // JniFrontend.buildTestDescriptorTable()
+
   jmethodID fe_ctor_;
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/testutil/desc-tbl-builder.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/desc-tbl-builder.cc b/be/src/testutil/desc-tbl-builder.cc
index 7793f06..77be724 100644
--- a/be/src/testutil/desc-tbl-builder.cc
+++ b/be/src/testutil/desc-tbl-builder.cc
@@ -19,13 +19,17 @@
 #include "util/bit-util.h"
 
 #include "common/object-pool.h"
+#include "service/frontend.h"
 #include "runtime/descriptors.h"
 
 #include "common/names.h"
 
 namespace impala {
 
-DescriptorTblBuilder::DescriptorTblBuilder(ObjectPool* obj_pool) : obj_pool_(obj_pool) {
+DescriptorTblBuilder::DescriptorTblBuilder(Frontend* fe, ObjectPool* obj_pool)
+  : fe_(fe), obj_pool_(obj_pool) {
+  DCHECK(fe != NULL);
+  DCHECK(obj_pool_ != NULL);
 }
 
 TupleDescBuilder& DescriptorTblBuilder::DeclareTuple() {
@@ -34,34 +38,6 @@ TupleDescBuilder& DescriptorTblBuilder::DeclareTuple() {
   return *tuple_builder;
 }
 
-// item_id of -1 indicates no itemTupleId
-static TSlotDescriptor MakeSlotDescriptor(int id, int parent_id, const ColumnType& type,
-    int slot_idx, int byte_offset, int item_id) {
-  int null_byte = slot_idx / 8;
-  int null_bit = slot_idx % 8;
-  TSlotDescriptor slot_desc;
-  slot_desc.__set_id(id);
-  slot_desc.__set_parent(parent_id);
-  slot_desc.__set_slotType(type.ToThrift());
-  slot_desc.__set_materializedPath(vector<int>(1, slot_idx));
-  slot_desc.__set_byteOffset(byte_offset);
-  slot_desc.__set_nullIndicatorByte(null_byte);
-  slot_desc.__set_nullIndicatorBit(null_bit);
-  slot_desc.__set_slotIdx(slot_idx);
-  if (item_id != -1) slot_desc.__set_itemTupleId(item_id);
-  return slot_desc;
-}
-
-static TTupleDescriptor MakeTupleDescriptor(int id, int byte_size, int num_null_bytes,
-    int table_id = -1) {
-  TTupleDescriptor tuple_desc;
-  tuple_desc.__set_id(id);
-  tuple_desc.__set_byteSize(byte_size);
-  tuple_desc.__set_numNullBytes(num_null_bytes);
-  if (table_id != -1) tuple_desc.__set_tableId(table_id);
-  return tuple_desc;
-}
-
 void DescriptorTblBuilder::SetTableDescriptor(const TTableDescriptor& table_desc) {
   DCHECK(thrift_desc_tbl_.tableDescriptors.empty())
       << "Only one TableDescriptor can be set.";
@@ -69,61 +45,25 @@ void DescriptorTblBuilder::SetTableDescriptor(const TTableDescriptor& table_desc
 }
 
 DescriptorTbl* DescriptorTblBuilder::Build() {
-  DescriptorTbl* desc_tbl;
-  int tuple_id = 0;
-  int slot_id = tuples_descs_.size(); // First ids reserved for TupleDescriptors
+  DCHECK(!tuples_descs_.empty());
 
+  TBuildTestDescriptorTableParams params;
   for (int i = 0; i < tuples_descs_.size(); ++i) {
-    BuildTuple(tuples_descs_[i]->slot_types(), &thrift_desc_tbl_, &tuple_id, &slot_id);
-  }
-
-  Status status = DescriptorTbl::Create(obj_pool_, thrift_desc_tbl_, &desc_tbl);
-  DCHECK(status.ok());
-  return desc_tbl;
-}
-
-TTupleDescriptor DescriptorTblBuilder::BuildTuple(
-    const vector<ColumnType>& slot_types, TDescriptorTable* thrift_desc_tbl,
-    int* next_tuple_id, int* slot_id) {
-  // We never materialize struct slots (there's no in-memory representation of structs,
-  // instead the materialized fields appear directly in the tuple), but array types can
-  // still have a struct item type. In this case, the array item tuple contains the
-  // "inlined" struct fields.
-  if (slot_types.size() == 1 && slot_types[0].type == TYPE_STRUCT) {
-    return BuildTuple(slot_types[0].children, thrift_desc_tbl, next_tuple_id, slot_id);
-  }
-
-  int num_null_bytes = BitUtil::Ceil(slot_types.size(), 8);
-  int byte_offset = num_null_bytes;
-  int tuple_id = *next_tuple_id;
-  ++(*next_tuple_id);
-
-  for (int i = 0; i < slot_types.size(); ++i) {
-    DCHECK_NE(slot_types[i].type, TYPE_STRUCT);
-    int item_id = -1;
-    if (slot_types[i].IsCollectionType()) {
-      TTupleDescriptor item_desc =
-          BuildTuple(slot_types[i].children, thrift_desc_tbl, next_tuple_id, slot_id);
-      item_id = item_desc.id;
+    params.slot_types.push_back(vector<TColumnType>());
+    vector<TColumnType>& tslot_types = params.slot_types.back();
+    const vector<ColumnType>& slot_types = tuples_descs_[i]->slot_types();
+    for (const ColumnType& slot_type : slot_types) {
+      tslot_types.push_back(slot_type.ToThrift());
     }
-
-    thrift_desc_tbl->slotDescriptors.push_back(
-        MakeSlotDescriptor(*slot_id, tuple_id, slot_types[i], i, byte_offset, item_id));
-    byte_offset += slot_types[i].GetSlotSize();
-    ++(*slot_id);
   }
 
-  TTupleDescriptor result;
+  Status buildDescTblStatus = fe_->BuildTestDescriptorTable(params, &thrift_desc_tbl_);
+  DCHECK(buildDescTblStatus.ok()) << buildDescTblStatus.GetDetail();
 
-  // If someone set a table descriptor pass that id along to the tuple descriptor.
-  if (thrift_desc_tbl_.tableDescriptors.empty()) {
-    result = MakeTupleDescriptor(tuple_id, byte_offset, num_null_bytes);
-  } else {
-    result = MakeTupleDescriptor(tuple_id, byte_offset, num_null_bytes,
-                                 thrift_desc_tbl_.tableDescriptors[0].id);
-  }
-  thrift_desc_tbl->tupleDescriptors.push_back(result);
-  return result;
+  DescriptorTbl* desc_tbl;
+  Status status = DescriptorTbl::Create(obj_pool_, thrift_desc_tbl_, &desc_tbl);
+  DCHECK(status.ok()) << status.GetDetail();
+  return desc_tbl;
 }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/be/src/testutil/desc-tbl-builder.h
----------------------------------------------------------------------
diff --git a/be/src/testutil/desc-tbl-builder.h b/be/src/testutil/desc-tbl-builder.h
index 8dc8a55..9ad92b1 100644
--- a/be/src/testutil/desc-tbl-builder.h
+++ b/be/src/testutil/desc-tbl-builder.h
@@ -28,18 +28,17 @@ class TupleDescBuilder;
 
 /// Aids in the construction of a DescriptorTbl by declaring tuples and slots
 /// associated with those tuples.
-/// TupleIds are monotonically increasing from 0 for each DeclareTuple, and
-/// SlotIds increase similarly, but are always greater than all TupleIds.
-/// Unlike FE, slots are not reordered based on size, and padding is not addded.
-//
+/// The descriptor table is constructed by calling into the FE via JNI, such that
+/// the tuple mem layouts mimic real queries. All id assignments happen in the FE.
+///
 /// Example usage:
 /// DescriptorTblBuilder builder;
-/// builder.DeclareTuple() << TYPE_TINYINT << TYPE_TIMESTAMP; // gets TupleId 0
-/// builder.DeclareTuple() << TYPE_FLOAT; // gets TupleId 1
+/// builder.DeclareTuple() << TYPE_TINYINT << TYPE_TIMESTAMP;
+/// builder.DeclareTuple() << TYPE_FLOAT;
 /// DescriptorTbl desc_tbl = builder.Build();
 class DescriptorTblBuilder {
  public:
-  DescriptorTblBuilder(ObjectPool* object_pool);
+  DescriptorTblBuilder(Frontend* fe, ObjectPool* object_pool);
 
   TupleDescBuilder& DeclareTuple();
 
@@ -50,15 +49,12 @@ class DescriptorTblBuilder {
   DescriptorTbl* Build();
 
  private:
-  /// Owned by caller.
+  /// Both owned by caller.
+  Frontend* fe_;
   ObjectPool* obj_pool_;
 
   std::vector<TupleDescBuilder*> tuples_descs_;
   TDescriptorTable thrift_desc_tbl_;
-
-  TTupleDescriptor BuildTuple(
-      const std::vector<ColumnType>& slot_types, TDescriptorTable* thrift_desc_tbl,
-      int* tuple_id, int* slot_id);
 };
 
 class TupleDescBuilder {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 91322b2..fbbf7be 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -749,3 +749,10 @@ struct TGetAllHadoopConfigsResponse {
 struct TStartupOptions {
   1: optional bool compute_lineage
 }
+
+// For creating a test descriptor table. The tuples and their memory layout are computed
+// in the FE.
+struct TBuildTestDescriptorTableParams {
+  // Every entry describes the slot types of one tuple.
+  1: required list<list<Types.TColumnType>> slot_types
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/fe/src/main/java/org/apache/impala/analysis/DescriptorTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/DescriptorTable.java b/fe/src/main/java/org/apache/impala/analysis/DescriptorTable.java
index 0f59fdb..22764ea 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DescriptorTable.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DescriptorTable.java
@@ -17,17 +17,22 @@
 
 package org.apache.impala.analysis;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.commons.lang.StringUtils;
-
+import org.apache.impala.catalog.ArrayType;
+import org.apache.impala.catalog.StructField;
+import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.View;
 import org.apache.impala.common.IdGenerator;
+import org.apache.impala.thrift.TColumnType;
 import org.apache.impala.thrift.TDescriptorTable;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -142,13 +147,13 @@ public class DescriptorTable {
     return result;
   }
 
-  // Computes physical layout parameters of all descriptors.
-  // Call this only after the last descriptor was added.
-  // Test-only.
+  /**
+   * Computes physical layout parameters of all descriptors.
+   * Call this only after the last descriptor was added.
+   * Test-only.
+   */
   public void computeMemLayout() {
-    for (TupleDescriptor d: tupleDescs_.values()) {
-      d.computeMemLayout();
-    }
+    for (TupleDescriptor d: tupleDescs_.values()) d.computeMemLayout();
   }
 
   public TDescriptorTable toThrift() {
@@ -195,4 +200,55 @@ public class DescriptorTable {
     }
     return out.toString();
   }
+
+  /**
+   * Creates a thrift descriptor table for testing. Each entry in 'slotTypes' is a list
+   * of slot types for one tuple.
+   */
+  public static TDescriptorTable buildTestDescriptorTable(
+      List<List<TColumnType>> slotTypes) {
+    DescriptorTable descTbl = new DescriptorTable();
+    for (List<TColumnType> ttupleSlots: slotTypes) {
+      ArrayList<StructField> fields = Lists.newArrayListWithCapacity(ttupleSlots.size());
+      for (TColumnType ttype: ttupleSlots) {
+        fields.add(new StructField("testField", Type.fromThrift(ttype)));
+      }
+      StructType tupleType = new StructType(fields);
+      createTupleDesc(tupleType, descTbl);
+    }
+    descTbl.computeMemLayout();
+    return descTbl.toThrift();
+  }
+
+  /**
+   * Recursive helper for buildTestDescriptorTable(). Returns a TupleDescriptor
+   * corresponding to the given struct. The struct may contain scalar and array fields.
+   */
+  private static TupleDescriptor createTupleDesc(StructType tupleType,
+      DescriptorTable descTbl) {
+    TupleDescriptor tupleDesc = descTbl.createTupleDescriptor("testDescTbl");
+    for (StructField field: tupleType.getFields()) {
+      Type type = field.getType();
+      SlotDescriptor slotDesc = descTbl.addSlotDescriptor(tupleDesc);
+      slotDesc.setIsMaterialized(true);
+      slotDesc.setType(type);
+      if (!type.isCollectionType()) continue;
+
+      // Set item tuple descriptor for the collection.
+      Preconditions.checkState(type.isArrayType());
+      ArrayType arrayType = (ArrayType) type;
+      Type itemType = arrayType.getItemType();
+      StructType itemStruct = null;
+      if (itemType.isStructType()) {
+        itemStruct = (StructType) itemType;
+      } else {
+        ArrayList<StructField> itemFields = Lists.newArrayListWithCapacity(1);
+        itemFields.add(new StructField("item", itemType));
+        itemStruct = new StructType(itemFields);
+      }
+      TupleDescriptor itemTuple = createTupleDesc(itemStruct, descTbl);
+      slotDesc.setItemTupleDesc(itemTuple);
+    }
+    return tupleDesc;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java b/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
index cbfdaca..e5462fd 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
@@ -24,13 +24,13 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
-
 import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.Table;
 import org.apache.impala.catalog.View;
 import org.apache.impala.thrift.TTupleDescriptor;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
@@ -56,6 +56,15 @@ import com.google.common.collect.Lists;
  * A tuple descriptor may be materialized or non-materialized. A non-materialized tuple
  * descriptor acts as a placeholder for 'virtual' table references such as inline views,
  * and must not be materialized at runtime.
+ *
+ * Memory Layout
+ * Slots are placed in descending order by size with trailing bytes to store null flags.
+ * Null flags are omitted for non-nullable slots. There is no padding between tuples when
+ * stored back-to-back in a row batch.
+ *
+ * Example: select bool_col, int_col, string_col, smallint_col from functional.alltypes
+ * Slots:   string_col|int_col|smallint_col|bool_col|null_byte
+ * Offsets: 0          16      20           22       23
  */
 public class TupleDescriptor {
   private final TupleId id_;
@@ -211,12 +220,13 @@ public class TupleDescriptor {
     if (hasMemLayout_) return;
     hasMemLayout_ = true;
 
-    // sort slots by size
+    // maps from slot size to slot descriptors with that size
     Map<Integer, List<SlotDescriptor>> slotsBySize =
         new HashMap<Integer, List<SlotDescriptor>>();
 
-    // populate slotsBySize; also compute avgSerializedSize
+    // populate slotsBySize
     int numNullableSlots = 0;
+    int totalSlotSize = 0;
     for (SlotDescriptor d: slots_) {
       if (!d.isMaterialized()) continue;
       ColumnStats stats = d.getStats();
@@ -229,6 +239,7 @@ public class TupleDescriptor {
       if (!slotsBySize.containsKey(d.getType().getSlotSize())) {
         slotsBySize.put(d.getType().getSlotSize(), new ArrayList<SlotDescriptor>());
       }
+      totalSlotSize += d.getType().getSlotSize();
       slotsBySize.get(d.getType().getSlotSize()).add(d);
       if (d.getIsNullable()) ++numNullableSlots;
     }
@@ -236,30 +247,25 @@ public class TupleDescriptor {
     Preconditions.checkState(!slotsBySize.containsKey(0));
     Preconditions.checkState(!slotsBySize.containsKey(-1));
 
-    // assign offsets to slots in order of ascending size
+    // assign offsets to slots in order of descending size
     numNullBytes_ = (numNullableSlots + 7) / 8;
-    int offset = numNullBytes_;
-    int nullIndicatorByte = 0;
+    int slotOffset = 0;
+    int nullIndicatorByte = totalSlotSize;
     int nullIndicatorBit = 0;
-    // slotIdx is the index into the resulting tuple struct.  The first (smallest) field
+    // slotIdx is the index into the resulting tuple struct.  The first (largest) field
     // is 0, next is 1, etc.
     int slotIdx = 0;
+    // sort slots in descending order of size
     List<Integer> sortedSizes = new ArrayList<Integer>(slotsBySize.keySet());
-    Collections.sort(sortedSizes);
+    Collections.sort(sortedSizes, Collections.reverseOrder());
     for (int slotSize: sortedSizes) {
       if (slotsBySize.get(slotSize).isEmpty()) continue;
-      if (slotSize > 1) {
-        // insert padding
-        int alignTo = Math.min(slotSize, 8);
-        offset = (offset + alignTo - 1) / alignTo * alignTo;
-      }
-
       for (SlotDescriptor d: slotsBySize.get(slotSize)) {
         Preconditions.checkState(d.isMaterialized());
         d.setByteSize(slotSize);
-        d.setByteOffset(offset);
+        d.setByteOffset(slotOffset);
         d.setSlotIdx(slotIdx++);
-        offset += slotSize;
+        slotOffset += slotSize;
 
         // assign null indicator
         if (d.getIsNullable()) {
@@ -268,14 +274,15 @@ public class TupleDescriptor {
           nullIndicatorBit = (nullIndicatorBit + 1) % 8;
           if (nullIndicatorBit == 0) ++nullIndicatorByte;
         } else {
-          // Non-nullable slots will have 0 for the byte offset and -1 for the bit mask
+          // non-nullable slots will have 0 for the byte offset and -1 for the bit mask
           d.setNullIndicatorBit(-1);
           d.setNullIndicatorByte(0);
         }
       }
     }
+    Preconditions.checkState(slotOffset == totalSlotSize);
 
-    this.byteSize_ = offset;
+    byteSize_ = totalSlotSize + numNullBytes_;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 0b4ce8f..07d6ec6 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -30,17 +30,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.log4j.Appender;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.log4j.FileAppender;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import org.apache.impala.analysis.DescriptorTable;
 import org.apache.impala.analysis.ToSqlUtils;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.ImpalaInternalAdminUser;
@@ -53,12 +46,13 @@ import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.JniUtil;
-import org.apache.impala.service.BackendConfig;
+import org.apache.impala.thrift.TBuildTestDescriptorTableParams;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TDatabase;
 import org.apache.impala.thrift.TDescribeDbParams;
 import org.apache.impala.thrift.TDescribeResult;
 import org.apache.impala.thrift.TDescribeTableParams;
+import org.apache.impala.thrift.TDescriptorTable;
 import org.apache.impala.thrift.TExecRequest;
 import org.apache.impala.thrift.TFunctionCategory;
 import org.apache.impala.thrift.TGetAllHadoopConfigsResponse;
@@ -90,6 +84,14 @@ import org.apache.impala.thrift.TUpdateMembershipRequest;
 import org.apache.impala.util.GlogAppender;
 import org.apache.impala.util.PatternMatcher;
 import org.apache.impala.util.TSessionStateUtil;
+import org.apache.log4j.Appender;
+import org.apache.log4j.FileAppender;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -477,6 +479,25 @@ public class JniFrontend {
   }
 
   /**
+   * Creates a thrift descriptor table for testing.
+   */
+  public byte[] buildTestDescriptorTable(byte[] buildTestDescTblParams)
+      throws ImpalaException {
+    TBuildTestDescriptorTableParams params = new TBuildTestDescriptorTableParams();
+    JniUtil.deserializeThrift(protocolFactory_, params, buildTestDescTblParams);
+    Preconditions.checkNotNull(params.slot_types);
+    TDescriptorTable result =
+        DescriptorTable.buildTestDescriptorTable(params.slot_types);
+    TSerializer serializer = new TSerializer(protocolFactory_);
+    try {
+      byte[] ret = serializer.serialize(result);
+      return ret;
+    } catch (TException e) {
+      throw new InternalException(e.getMessage());
+    }
+  }
+
+  /**
    * Gets all roles
    */
   public byte[] getRoles(byte[] showRolesParams) throws ImpalaException {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b0e87c68/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
index d763deb..993f489 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
@@ -22,18 +22,17 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.impala.catalog.Function;
-import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FrontendTestBase;
 import org.apache.impala.thrift.TExpr;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 
 public class AnalyzerTest extends FrontendTestBase {
@@ -173,27 +172,24 @@ public class AnalyzerTest extends FrontendTestBase {
     SelectStmt stmt = (SelectStmt) AnalyzesOk("select * from functional.AllTypes");
     Analyzer analyzer = stmt.getAnalyzer();
     DescriptorTable descTbl = analyzer.getDescTbl();
-    TupleDescriptor tupleD = descTbl.getTupleDesc(new TupleId(0));
-    for (SlotDescriptor slotD: tupleD.getSlots()) {
-      slotD.setIsMaterialized(true);
-    }
+    TupleDescriptor tupleDesc = descTbl.getTupleDesc(new TupleId(0));
+    tupleDesc.materializeSlots();
     descTbl.computeMemLayout();
-    Assert.assertEquals(97.0f, tupleD.getAvgSerializedSize(), 0.0);
-    checkLayoutParams("functional.alltypes.bool_col", 1, 2, 0, 0, analyzer);
-    checkLayoutParams("functional.alltypes.tinyint_col", 1, 3, 0, 1, analyzer);
-    checkLayoutParams("functional.alltypes.smallint_col", 2, 4, 0, 2, analyzer);
-    checkLayoutParams("functional.alltypes.id", 4, 8, 0, 3, analyzer);
-    checkLayoutParams("functional.alltypes.int_col", 4, 12, 0, 4, analyzer);
-    checkLayoutParams("functional.alltypes.float_col", 4, 16, 0, 5, analyzer);
-    checkLayoutParams("functional.alltypes.year", 4, 20, 0, 6, analyzer);
-    checkLayoutParams("functional.alltypes.month", 4, 24, 0, 7, analyzer);
-    checkLayoutParams("functional.alltypes.bigint_col", 8, 32, 1, 0, analyzer);
-    checkLayoutParams("functional.alltypes.double_col", 8, 40, 1, 1, analyzer);
-    int strSlotSize = PrimitiveType.STRING.getSlotSize();
-    checkLayoutParams("functional.alltypes.date_string_col",
-        strSlotSize, 48, 1, 2, analyzer);
-    checkLayoutParams("functional.alltypes.string_col",
-        strSlotSize, 48 + strSlotSize, 1, 3, analyzer);
+
+    Assert.assertEquals(97.0f, tupleDesc.getAvgSerializedSize(), 0.0);
+    checkLayoutParams("functional.alltypes.date_string_col", 16, 0, 88, 0, analyzer);
+    checkLayoutParams("functional.alltypes.string_col", 16, 16, 88, 1, analyzer);
+    checkLayoutParams("functional.alltypes.timestamp_col", 16, 32, 88, 2, analyzer);
+    checkLayoutParams("functional.alltypes.bigint_col", 8, 48, 88, 3, analyzer);
+    checkLayoutParams("functional.alltypes.double_col", 8, 56, 88, 4, analyzer);
+    checkLayoutParams("functional.alltypes.id", 4, 64, 88, 5, analyzer);
+    checkLayoutParams("functional.alltypes.int_col", 4, 68, 88, 6, analyzer);
+    checkLayoutParams("functional.alltypes.float_col", 4, 72, 88, 7, analyzer);
+    checkLayoutParams("functional.alltypes.year", 4, 76, 89, 0, analyzer);
+    checkLayoutParams("functional.alltypes.month", 4, 80, 89, 1, analyzer);
+    checkLayoutParams("functional.alltypes.smallint_col", 2, 84, 89, 2, analyzer);
+    checkLayoutParams("functional.alltypes.bool_col", 1, 86, 89, 3, analyzer);
+    checkLayoutParams("functional.alltypes.tinyint_col", 1, 87, 89, 4, analyzer);
   }
 
   private void testNonNullable() throws AnalysisException {
@@ -205,9 +201,7 @@ public class AnalyzerTest extends FrontendTestBase {
         "select count(int_col), count(*) from functional.AllTypes");
     DescriptorTable descTbl = stmt.getAnalyzer().getDescTbl();
     TupleDescriptor aggDesc = descTbl.getTupleDesc(new TupleId(1));
-    for (SlotDescriptor slotD: aggDesc.getSlots()) {
-      slotD.setIsMaterialized(true);
-    }
+    aggDesc.materializeSlots();
     descTbl.computeMemLayout();
     Assert.assertEquals(16.0f, aggDesc.getAvgSerializedSize(), 0.0);
     Assert.assertEquals(16, aggDesc.getByteSize());
@@ -218,22 +212,19 @@ public class AnalyzerTest extends FrontendTestBase {
   private void testMixedNullable() throws AnalysisException {
     // one slot is nullable, one is not. The layout should look like:
     // (byte range : data)
-    // 0 : 1 nullable-byte (only 1 bit used)
-    // 1 - 7: padded bytes
-    // 8 - 15: sum(int_col)
-    // 16 - 23: count(*)
+    // 0 - 7: sum(int_col)
+    // 8 - 15: count(*)
+    // 16 - 17: nullable-byte (only 1 bit used)
     SelectStmt stmt = (SelectStmt) AnalyzesOk(
         "select sum(int_col), count(*) from functional.AllTypes");
     DescriptorTable descTbl = stmt.getAnalyzer().getDescTbl();
     TupleDescriptor aggDesc = descTbl.getTupleDesc(new TupleId(1));
-    for (SlotDescriptor slotD: aggDesc.getSlots()) {
-      slotD.setIsMaterialized(true);
-    }
+    aggDesc.materializeSlots();
     descTbl.computeMemLayout();
     Assert.assertEquals(16.0f, aggDesc.getAvgSerializedSize(), 0.0);
-    Assert.assertEquals(24, aggDesc.getByteSize());
-    checkLayoutParams(aggDesc.getSlots().get(0), 8, 8, 0, 0);
-    checkLayoutParams(aggDesc.getSlots().get(1), 8, 16, 0, -1);
+    Assert.assertEquals(17, aggDesc.getByteSize());
+    checkLayoutParams(aggDesc.getSlots().get(0), 8, 0, 16, 0);
+    checkLayoutParams(aggDesc.getSlots().get(1), 8, 8, 0, -1);
   }
 
   /**
@@ -243,34 +234,31 @@ public class AnalyzerTest extends FrontendTestBase {
     SelectStmt stmt = (SelectStmt) AnalyzesOk("select * from functional.alltypes");
     Analyzer analyzer = stmt.getAnalyzer();
     DescriptorTable descTbl = analyzer.getDescTbl();
-    TupleDescriptor tupleD = descTbl.getTupleDesc(new TupleId(0));
-    ArrayList<SlotDescriptor> slots = tupleD.getSlots();
-    for (SlotDescriptor slotD: slots) {
-      slotD.setIsMaterialized(true);
-    }
+    TupleDescriptor tupleDesc = descTbl.getTupleDesc(new TupleId(0));
+    tupleDesc.materializeSlots();
     // Mark slots 0 (id), 7 (double_col), 9 (string_col) as non-materialized.
+    ArrayList<SlotDescriptor> slots = tupleDesc.getSlots();
     slots.get(0).setIsMaterialized(false);
     slots.get(7).setIsMaterialized(false);
     slots.get(9).setIsMaterialized(false);
-
     descTbl.computeMemLayout();
-    Assert.assertEquals(68.0f, tupleD.getAvgSerializedSize(), 0.0);
+
+    Assert.assertEquals(68.0f, tupleDesc.getAvgSerializedSize(), 0.0);
     // Check non-materialized slots.
     checkLayoutParams("functional.alltypes.id", 0, -1, 0, 0, analyzer);
     checkLayoutParams("functional.alltypes.double_col", 0, -1, 0, 0, analyzer);
     checkLayoutParams("functional.alltypes.string_col", 0, -1, 0, 0, analyzer);
     // Check materialized slots.
-    checkLayoutParams("functional.alltypes.bool_col", 1, 2, 0, 0, analyzer);
-    checkLayoutParams("functional.alltypes.tinyint_col", 1, 3, 0, 1, analyzer);
-    checkLayoutParams("functional.alltypes.smallint_col", 2, 4, 0, 2, analyzer);
-    checkLayoutParams("functional.alltypes.int_col", 4, 8, 0, 3, analyzer);
-    checkLayoutParams("functional.alltypes.float_col", 4, 12, 0, 4, analyzer);
-    checkLayoutParams("functional.alltypes.year", 4, 16, 0, 5, analyzer);
-    checkLayoutParams("functional.alltypes.month", 4, 20, 0, 6, analyzer);
-    checkLayoutParams("functional.alltypes.bigint_col", 8, 24, 0, 7, analyzer);
-    int strSlotSize = PrimitiveType.STRING.getSlotSize();
-    checkLayoutParams("functional.alltypes.date_string_col",
-        strSlotSize, 32, 1, 0, analyzer);
+    checkLayoutParams("functional.alltypes.date_string_col", 16, 0, 60, 0, analyzer);
+    checkLayoutParams("functional.alltypes.timestamp_col", 16, 16, 60, 1, analyzer);
+    checkLayoutParams("functional.alltypes.bigint_col", 8, 32, 60, 2, analyzer);
+    checkLayoutParams("functional.alltypes.int_col", 4, 40, 60, 3, analyzer);
+    checkLayoutParams("functional.alltypes.float_col", 4, 44, 60, 4, analyzer);
+    checkLayoutParams("functional.alltypes.year", 4, 48, 60, 5, analyzer);
+    checkLayoutParams("functional.alltypes.month", 4, 52, 60, 6, analyzer);
+    checkLayoutParams("functional.alltypes.smallint_col", 2, 56, 60, 7, analyzer);
+    checkLayoutParams("functional.alltypes.bool_col", 1, 58, 61, 0, analyzer);
+    checkLayoutParams("functional.alltypes.tinyint_col", 1, 59, 61, 1, analyzer);
   }
 
   private void checkLayoutParams(SlotDescriptor d, int byteSize, int byteOffset,