You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jr...@apache.org on 2017/03/12 18:27:03 UTC

[2/2] incubator-impala git commit: IMPALA-4008: Don't bake fields into generated IR functions of OldHashTable

IMPALA-4008: Don't bake fields into generated IR functions of OldHashTable

To support sharing generated code across fragment instances,
no fragment instance specific states should be baked into the
IR. All cases were addressed previously except for the old hash
tables used for legacy agg/join. This change fixes the old hash
tables to not bake its fields in the generated IR functions. Similar
to previous patches, some cross-compiled thin wrappers are introduced
to access the fields of interest from an OldHashTable object.

Change-Id: I75500827dff56b1fa9e5296e8e8d8667ab54aef8
Reviewed-on: http://gerrit.cloudera.org:8080/6263
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 56e37166492b8ee155a6bae851489ace635ae085
Parents: 244ec22
Author: Michael Ho <kw...@cloudera.com>
Authored: Tue Mar 7 22:09:52 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sat Mar 11 06:40:17 2017 +0000

----------------------------------------------------------------------
 be/src/codegen/gen_ir_descriptions.py |  12 ++-
 be/src/codegen/impala-ir.cc           |   1 +
 be/src/codegen/llvm-codegen.cc        |   7 ++
 be/src/codegen/llvm-codegen.h         |   5 +
 be/src/exec/CMakeLists.txt            |   1 +
 be/src/exec/hash-table-ir.cc          |   8 +-
 be/src/exec/hash-table.cc             |  34 +++----
 be/src/exec/hash-table.h              |   5 +-
 be/src/exec/old-hash-table-ir.cc      |  42 +++++++++
 be/src/exec/old-hash-table.cc         | 141 ++++++++++++++++++-----------
 be/src/exec/old-hash-table.h          |   8 ++
 11 files changed, 184 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/codegen/gen_ir_descriptions.py
----------------------------------------------------------------------
diff --git a/be/src/codegen/gen_ir_descriptions.py b/be/src/codegen/gen_ir_descriptions.py
index 44fa3cf..4b62dfe 100755
--- a/be/src/codegen/gen_ir_descriptions.py
+++ b/be/src/codegen/gen_ir_descriptions.py
@@ -108,6 +108,14 @@ ir_functions = [
    "_ZN6impala12HashJoinNode17ProcessBuildBatchEPNS_8RowBatchE"],
   ["HASH_JOIN_PROCESS_PROBE_BATCH",
    "_ZN6impala12HashJoinNode17ProcessProbeBatchEPNS_8RowBatchES2_i"],
+  ["OLD_HASH_TABLE_GET_BUILD_EXPR_CTXS",
+   "_ZNK6impala12OldHashTable15build_expr_ctxsEv"],
+  ["OLD_HASH_TABLE_GET_PROBE_EXPR_CTXS",
+   "_ZNK6impala12OldHashTable15probe_expr_ctxsEv"],
+  ["OLD_HASH_TABLE_GET_EXPR_VALUES_BUFFER",
+   "_ZNK6impala12OldHashTable18expr_values_bufferEv"],
+  ["OLD_HASH_TABLE_GET_EXPR_VALUE_NULL_BITS",
+   "_ZNK6impala12OldHashTable20expr_value_null_bitsEv"],
   ["PHJ_PROCESS_BUILD_BATCH",
    "_ZN6impala10PhjBuilder17ProcessBuildBatchEPNS_8RowBatchEPNS_12HashTableCtxEb"],
   ["PHJ_PROCESS_PROBE_BATCH_INNER_JOIN",
@@ -133,9 +141,9 @@ ir_functions = [
   ["HASH_TABLE_GET_HASH_SEED",
    "_ZNK6impala12HashTableCtx11GetHashSeedEv"],
   ["HASH_TABLE_GET_BUILD_EXPR_CTX",
-   "_ZNK6impala12HashTableCtx15GetBuildExprCtxEi"],
+   "_ZNK6impala12HashTableCtx16GetBuildExprCtxsEv"],
   ["HASH_TABLE_GET_PROBE_EXPR_CTX",
-   "_ZNK6impala12HashTableCtx15GetProbeExprCtxEi"],
+   "_ZNK6impala12HashTableCtx16GetProbeExprCtxsEv"],
   ["HLL_UPDATE_BOOLEAN",
    "_ZN6impala18AggregateFunctions9HllUpdateIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE"],
   ["HLL_UPDATE_TINYINT",

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/codegen/impala-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/impala-ir.cc b/be/src/codegen/impala-ir.cc
index 2ef8790..5ff816b 100644
--- a/be/src/codegen/impala-ir.cc
+++ b/be/src/codegen/impala-ir.cc
@@ -32,6 +32,7 @@
 #include "exec/hdfs-avro-scanner-ir.cc"
 #include "exec/hdfs-parquet-scanner-ir.cc"
 #include "exec/hdfs-scanner-ir.cc"
+#include "exec/old-hash-table-ir.cc"
 #include "exec/partitioned-aggregation-node-ir.cc"
 #include "exec/partitioned-hash-join-builder-ir.cc"
 #include "exec/partitioned-hash-join-node-ir.cc"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 7977c01..4ad5c93 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -1406,6 +1406,13 @@ Value* LlvmCodeGen::CodegenArrayAt(LlvmBuilder* builder, Value* array, int idx,
   return builder->CreateLoad(ptr, name);
 }
 
+Value* LlvmCodeGen::CodegenCallFunction(LlvmBuilder* builder,
+    IRFunction::Type ir_type, ArrayRef<Value*> args, const char* name) {
+  Function* fn = GetFunction(ir_type, false);
+  DCHECK(fn != nullptr);
+  return builder->CreateCall(fn, args, name);
+}
+
 void LlvmCodeGen::ClearHashFns() {
   hash_fns_.clear();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 5a8c494..7259081 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -505,6 +505,11 @@ class LlvmCodeGen {
   llvm::Value* CodegenArrayAt(
       LlvmBuilder*, llvm::Value* array, int idx, const char* name = "");
 
+  /// Codegens IR to call the function corresponding to 'ir_type' with argument 'args'
+  /// and returns the value.
+  llvm::Value* CodegenCallFunction(LlvmBuilder* builder, IRFunction::Type ir_type,
+      llvm::ArrayRef<llvm::Value*> args, const char* name);
+
   /// If there are more than this number of expr trees (or functions that evaluate
   /// expressions), avoid inlining avoid inlining for the exprs exceeding this threshold.
   static const int CODEGEN_INLINE_EXPRS_THRESHOLD = 100;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 57c12cb..3d236a8 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -42,6 +42,7 @@ add_library(Exec
   hash-join-node.cc
   hash-join-node-ir.cc
   old-hash-table.cc
+  old-hash-table-ir.cc
   hash-table.cc
   hbase-table-sink.cc
   hbase-table-writer.cc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/hash-table-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table-ir.cc b/be/src/exec/hash-table-ir.cc
index a702736..4da81e2 100644
--- a/be/src/exec/hash-table-ir.cc
+++ b/be/src/exec/hash-table-ir.cc
@@ -23,8 +23,12 @@ using namespace impala;
 
 uint32_t HashTableCtx::GetHashSeed() const { return seeds_[level_]; }
 
-ExprContext* HashTableCtx::GetBuildExprCtx(int i) const { return build_expr_ctxs_[i]; }
+ExprContext* const* HashTableCtx::GetBuildExprCtxs() const {
+  return build_expr_ctxs_.data();
+}
 
-ExprContext* HashTableCtx::GetProbeExprCtx(int i) const { return probe_expr_ctxs_[i]; }
+ExprContext* const* HashTableCtx::GetProbeExprCtxs() const {
+  return probe_expr_ctxs_.data();
+}
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index f4ecb67..d8a7cdc 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -728,11 +728,11 @@ Status HashTableCtx::CodegenEvalRow(LlvmCodeGen* codegen, bool build, Function**
   Value* expr_values_null = args[3];
   Value* has_null = codegen->false_value();
 
-  IRFunction::Type get_expr_ctx_fn_name = build ?
+  // ctx_vector = &build_expr_ctxs_[0] / ctx_vector = &probe_expr_ctxs_[0]
+  Value* ctx_vector = codegen->CodegenCallFunction(&builder, build ?
       IRFunction::HASH_TABLE_GET_BUILD_EXPR_CTX :
-      IRFunction::HASH_TABLE_GET_PROBE_EXPR_CTX;
-  Function* get_expr_ctx_fn = codegen->GetFunction(get_expr_ctx_fn_name, false);
-  DCHECK(get_expr_ctx_fn != NULL);
+      IRFunction::HASH_TABLE_GET_PROBE_EXPR_CTX,
+      this_ptr, "ctx_vector");
 
   for (int i = 0; i < ctxs.size(); ++i) {
     // TODO: refactor this to somewhere else?  This is not hash table specific except for
@@ -764,11 +764,9 @@ Status HashTableCtx::CodegenEvalRow(LlvmCodeGen* codegen, bool build, Function**
       codegen->SetNoInline(expr_fn);
     }
 
-    Value* get_expr_ctx_args[] = {this_ptr, codegen->GetIntConstant(TYPE_INT, i)};
-    Value* ctx_arg = builder.CreateCall(get_expr_ctx_fn, get_expr_ctx_args, "expr_ctx");
-    Value* expr_fn_args[] = {ctx_arg, row};
+    Value* expr_ctx = codegen->CodegenArrayAt(&builder, ctx_vector, i, "expr_ctx");
     CodegenAnyVal result = CodegenAnyVal::CreateCallWrapped(
-        codegen, &builder, ctxs[i]->root()->type(), expr_fn, expr_fn_args, "result");
+        codegen, &builder, ctxs[i]->root()->type(), expr_fn, {expr_ctx, row}, "result");
     Value* is_null = result.GetIsNull();
 
     // Set null-byte result
@@ -881,10 +879,8 @@ Status HashTableCtx::CodegenHashRow(LlvmCodeGen* codegen, bool use_murmur, Funct
   Value* expr_values_null = args[2];
 
   // Call GetHashSeed() to get seeds_[level_]
-  Function* get_hash_seed_fn =
-      codegen->GetFunction(IRFunction::HASH_TABLE_GET_HASH_SEED, false);
-  Value* seed = builder.CreateCall(get_hash_seed_fn, ArrayRef<Value*>({this_arg}),
-      "seed");
+  Value* seed = codegen->CodegenCallFunction(&builder,
+      IRFunction::HASH_TABLE_GET_HASH_SEED, this_arg, "seed");
 
   Value* hash_result = seed;
   const int var_result_offset = expr_values_cache_.var_result_offset();
@@ -1094,9 +1090,9 @@ Status HashTableCtx::CodegenEquals(LlvmCodeGen* codegen, bool force_null_equalit
   Value* expr_values = args[2];
   Value* expr_values_null = args[3];
 
-  Function* get_expr_ctx_fn =
-      codegen->GetFunction(IRFunction::HASH_TABLE_GET_BUILD_EXPR_CTX, false);
-  DCHECK(get_expr_ctx_fn != NULL);
+  // ctx_vector = &build_expr_ctxs_[0]
+  Value* ctx_vector = codegen->CodegenCallFunction(&builder,
+      IRFunction::HASH_TABLE_GET_BUILD_EXPR_CTX, this_ptr, "ctx_vector");
 
   BasicBlock* false_block = BasicBlock::Create(context, "false_block", *fn);
   for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
@@ -1118,14 +1114,12 @@ Status HashTableCtx::CodegenEquals(LlvmCodeGen* codegen, bool force_null_equalit
       codegen->SetNoInline(expr_fn);
     }
 
-    // Load ExprContext* from 'build_expr_ctxs_'.
-    Value* get_expr_ctx_args[] = {this_ptr, codegen->GetIntConstant(TYPE_INT, i)};
-    Value* ctx_arg = builder.CreateCall(get_expr_ctx_fn, get_expr_ctx_args, "expr_ctx");
+    // Load ExprContext*: expr_ctx = ctx_vector[i];
+    Value* expr_ctx = codegen->CodegenArrayAt(&builder, ctx_vector, i, "expr_ctx");
 
     // Evaluate the expression.
-    Value* expr_fn_args[] = { ctx_arg, row };
     CodegenAnyVal result = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
-        build_expr_ctxs_[i]->root()->type(), expr_fn, expr_fn_args, "result");
+        build_expr_ctxs_[i]->root()->type(), expr_fn, {expr_ctx, row}, "result");
     Value* is_null = result.GetIsNull();
 
     // Determine if row is null (i.e. expr_values_null[i] == true). In

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index 300b9e7..a93d601 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -463,8 +463,9 @@ class HashTableCtx {
   bool IR_NO_INLINE finds_some_nulls() const { return finds_some_nulls_; }
 
   /// Cross-compiled function to access the build/probe expression context.
-  ExprContext* IR_ALWAYS_INLINE GetBuildExprCtx(int i) const;
-  ExprContext* IR_ALWAYS_INLINE GetProbeExprCtx(int i) const;
+  /// Called by generated LLVM IR functions such as Equals() and EvalRow().
+  ExprContext* const* IR_ALWAYS_INLINE GetBuildExprCtxs() const;
+  ExprContext* const* IR_ALWAYS_INLINE GetProbeExprCtxs() const;
 
   const std::vector<ExprContext*>& build_expr_ctxs_;
   const std::vector<ExprContext*>& probe_expr_ctxs_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/old-hash-table-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table-ir.cc b/be/src/exec/old-hash-table-ir.cc
new file mode 100644
index 0000000..ef94892
--- /dev/null
+++ b/be/src/exec/old-hash-table-ir.cc
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifdef IR_COMPILE
+
+#include "exec/old-hash-table.h"
+
+namespace impala {
+
+uint8_t* OldHashTable::expr_values_buffer() const {
+  return expr_values_buffer_;
+}
+
+uint8_t* OldHashTable::expr_value_null_bits() const {
+  return expr_value_null_bits_;
+}
+
+ExprContext* const* OldHashTable::build_expr_ctxs() const {
+  return &build_expr_ctxs_[0];
+}
+
+ExprContext* const* OldHashTable::probe_expr_ctxs() const {
+  return &probe_expr_ctxs_[0];
+}
+
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/old-hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table.cc b/be/src/exec/old-hash-table.cc
index 44bf7e4..87b2e49 100644
--- a/be/src/exec/old-hash-table.cc
+++ b/be/src/exec/old-hash-table.cc
@@ -291,28 +291,41 @@ Function* OldHashTable::CodegenEvalTupleRow(LlvmCodeGen* codegen, bool build) {
   LlvmBuilder builder(context);
   Value* args[2];
   Function* fn = prototype.GeneratePrototype(&builder, args);
-
+  Value* this_ptr = args[0];
   Value* row = args[1];
   Value* has_null = codegen->false_value();
 
   // Aggregation with no grouping exprs also use the hash table interface for
   // code simplicity.  In that case, there are no build exprs.
   if (!build_expr_ctxs_.empty()) {
-    const vector<ExprContext*>& ctxs = build ? build_expr_ctxs_ : probe_expr_ctxs_;
-    for (int i = 0; i < ctxs.size(); ++i) {
-      // TODO: refactor this to somewhere else?  This is not hash table specific
-      // except for the null handling bit and would be used for anyone that needs
-      // to materialize a vector of exprs
-      // Convert result buffer to llvm ptr type
-      void* loc = expr_values_buffer_ + expr_values_buffer_offsets_[i];
-      Value* llvm_loc = codegen->CastPtrToLlvmPtr(
-          codegen->GetPtrType(ctxs[i]->root()->type()), loc);
+    // Load &build_expr_ctxs_[0] / &probe_expr_ctxs_[0]
+    Value* ctx_array = codegen->CodegenCallFunction(&builder, build ?
+        IRFunction::OLD_HASH_TABLE_GET_BUILD_EXPR_CTXS :
+        IRFunction::OLD_HASH_TABLE_GET_PROBE_EXPR_CTXS,
+        this_ptr, "ctx_array");
+
+    // Load expr_values_buffer_
+    Value* expr_values_buffer = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUES_BUFFER,
+        this_ptr, "expr_values_buffer");
+
+    // Load expr_values_null_bits_
+    Value* expr_value_null_bits = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUE_NULL_BITS,
+        this_ptr, "expr_value_null_bits");
 
+    for (int i = 0; i < ctxs.size(); ++i) {
       BasicBlock* null_block = BasicBlock::Create(context, "null", fn);
       BasicBlock* not_null_block = BasicBlock::Create(context, "not_null", fn);
       BasicBlock* continue_block = BasicBlock::Create(context, "continue", fn);
 
-      // Call expr
+      // loc_addr = expr_values_buffer_ + expr_values_buffer_offsets_[i]
+      Value* llvm_loc = builder.CreateInBoundsGEP(NULL, expr_values_buffer,
+          codegen->GetIntConstant(TYPE_INT, expr_values_buffer_offsets_[i]), "loc_addr");
+      llvm_loc = builder.CreatePointerCast(llvm_loc,
+          codegen->GetPtrType(ctxs[i]->root()->type()), "loc");
+
+      // Codegen GetValue() for ctxs[i]
       Function* expr_fn;
       Status status = ctxs[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
       if (!status.ok()) {
@@ -321,20 +334,18 @@ Function* OldHashTable::CodegenEvalTupleRow(LlvmCodeGen* codegen, bool build) {
         return NULL;
       }
 
-      Value* ctx_arg = codegen->CastPtrToLlvmPtr(
-          codegen->GetPtrType(ExprContext::LLVM_CLASS_NAME), ctxs[i]);
-      Value* expr_fn_args[] = { ctx_arg, row };
+      // Load ctxs[i] and call GetValue()
+      Value* expr_ctx = codegen->CodegenArrayAt(&builder, ctx_array, i, "expr_ctx");
+      DCHECK(expr_ctx->getType()->isPointerTy());
       CodegenAnyVal result = CodegenAnyVal::CreateCallWrapped(
-          codegen, &builder, ctxs[i]->root()->type(), expr_fn, expr_fn_args, "result");
+          codegen, &builder, ctxs[i]->root()->type(), expr_fn, {expr_ctx, row}, "result");
       Value* is_null = result.GetIsNull();
 
       // Set null-byte result
-      Value* null_byte = builder.CreateZExt(is_null, codegen->GetType(TYPE_TINYINT));
-      uint8_t* null_byte_loc = &expr_value_null_bits_[i];
-      Value* llvm_null_byte_loc =
-          codegen->CastPtrToLlvmPtr(codegen->ptr_type(), null_byte_loc);
-      builder.CreateStore(null_byte, llvm_null_byte_loc);
-
+      Value* null_bits = builder.CreateZExt(is_null, codegen->GetType(TYPE_TINYINT));
+      Value* llvm_null_bits_loc = builder.CreateInBoundsGEP(NULL, expr_value_null_bits,
+          codegen->GetIntConstant(TYPE_INT, i), "null_bits_loc");
+      builder.CreateStore(null_bits, llvm_null_bits_loc);
       builder.CreateCondBr(is_null, null_block, not_null_block);
 
       // Null block
@@ -430,38 +441,47 @@ Function* OldHashTable::CodegenHashCurrentRow(LlvmCodeGen* codegen) {
 
   LLVMContext& context = codegen->context();
   LlvmBuilder builder(context);
-  Value* this_arg;
-  Function* fn = prototype.GeneratePrototype(&builder, &this_arg);
+  Value* this_ptr;
+  Function* fn = prototype.GeneratePrototype(&builder, &this_ptr);
+
+  // Load expr_values_buffer_
+  Value* expr_values_buffer = codegen->CodegenCallFunction(&builder,
+      IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUES_BUFFER, this_ptr, "expr_values_buffer");
 
   Value* hash_result = codegen->GetIntConstant(TYPE_INT, initial_seed_);
-  Value* data = codegen->CastPtrToLlvmPtr(codegen->ptr_type(), expr_values_buffer_);
   if (var_result_begin_ == -1) {
     // No variable length slots, just hash what is in 'expr_values_buffer_'
     if (results_buffer_size_ > 0) {
       Function* hash_fn = codegen->GetHashFunction(results_buffer_size_);
       Value* len = codegen->GetIntConstant(TYPE_INT, results_buffer_size_);
-      hash_result =
-          builder.CreateCall(hash_fn, ArrayRef<Value*>({data, len, hash_result}));
+      hash_result = builder.CreateCall(hash_fn, {expr_values_buffer, len, hash_result});
     }
   } else {
     if (var_result_begin_ > 0) {
       Function* hash_fn = codegen->GetHashFunction(var_result_begin_);
       Value* len = codegen->GetIntConstant(TYPE_INT, var_result_begin_);
-      hash_result =
-          builder.CreateCall(hash_fn, ArrayRef<Value*>({data, len, hash_result}));
+      hash_result = builder.CreateCall(hash_fn, {expr_values_buffer, len, hash_result});
     }
 
+    // Load expr_value_null_bits_
+    Value* expr_value_null_bits = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUE_NULL_BITS,
+        this_ptr, "expr_value_null_bits");
+
     // Hash string slots
     for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
-      if (build_expr_ctxs_[i]->root()->type().type != TYPE_STRING
-          && build_expr_ctxs_[i]->root()->type().type != TYPE_VARCHAR) continue;
+      if (build_expr_ctxs_[i]->root()->type().type != TYPE_STRING &&
+          build_expr_ctxs_[i]->root()->type().type != TYPE_VARCHAR) {
+        continue;
+      }
 
       BasicBlock* null_block = NULL;
       BasicBlock* not_null_block = NULL;
       BasicBlock* continue_block = NULL;
       Value* str_null_result = NULL;
 
-      void* loc = expr_values_buffer_ + expr_values_buffer_offsets_[i];
+      Value* llvm_buffer_loc = builder.CreateInBoundsGEP(NULL, expr_values_buffer,
+          codegen->GetIntConstant(TYPE_INT, expr_values_buffer_offsets_[i]), "buffer_loc");
 
       // If the hash table stores nulls, we need to check if the stringval
       // evaluated to NULL
@@ -470,29 +490,28 @@ Function* OldHashTable::CodegenHashCurrentRow(LlvmCodeGen* codegen) {
         not_null_block = BasicBlock::Create(context, "not_null", fn);
         continue_block = BasicBlock::Create(context, "continue", fn);
 
-        uint8_t* null_byte_loc = &expr_value_null_bits_[i];
-        Value* llvm_null_byte_loc =
-            codegen->CastPtrToLlvmPtr(codegen->ptr_type(), null_byte_loc);
-        Value* null_byte = builder.CreateLoad(llvm_null_byte_loc);
-        Value* is_null = builder.CreateICmpNE(null_byte,
+        // Load expr_values_null_bits_[i] and check if it's set.
+        Value* llvm_null_bits_loc = builder.CreateInBoundsGEP(NULL, expr_value_null_bits,
+            codegen->GetIntConstant(TYPE_INT, i), "null_bits_loc");
+        Value* null_bits = builder.CreateLoad(llvm_null_bits_loc);
+        Value* is_null = builder.CreateICmpNE(null_bits,
             codegen->GetIntConstant(TYPE_TINYINT, 0));
         builder.CreateCondBr(is_null, null_block, not_null_block);
 
-        // For null, we just want to call the hash function on the portion of
-        // the data
+        // For null, we just want to call the hash function on a portion of the data.
         builder.SetInsertPoint(null_block);
         Function* null_hash_fn = codegen->GetHashFunction(sizeof(StringValue));
-        Value* llvm_loc = codegen->CastPtrToLlvmPtr(codegen->ptr_type(), loc);
         Value* len = codegen->GetIntConstant(TYPE_INT, sizeof(StringValue));
         str_null_result = builder.CreateCall(null_hash_fn,
-            ArrayRef<Value*>({llvm_loc, len, hash_result}));
+            ArrayRef<Value*>({llvm_buffer_loc, len, hash_result}));
         builder.CreateBr(continue_block);
 
         builder.SetInsertPoint(not_null_block);
       }
 
       // Convert expr_values_buffer_ loc to llvm value
-      Value* str_val = codegen->CastPtrToLlvmPtr(codegen->GetPtrType(TYPE_STRING), loc);
+      Value* str_val = builder.CreatePointerCast(llvm_buffer_loc,
+          codegen->GetPtrType(TYPE_STRING), "str_val");
 
       Value* ptr = builder.CreateStructGEP(NULL, str_val, 0, "ptr");
       Value* len = builder.CreateStructGEP(NULL, str_val, 1, "len");
@@ -615,17 +634,32 @@ Function* OldHashTable::CodegenEquals(LlvmCodeGen* codegen) {
   LlvmBuilder builder(context);
   Value* args[2];
   Function* fn = prototype.GeneratePrototype(&builder, args);
+  Value* this_ptr = args[0];
   Value* row = args[1];
 
   if (!build_expr_ctxs_.empty()) {
     BasicBlock* false_block = BasicBlock::Create(context, "false_block", fn);
 
+    // Load &build_expr_ctxs_[0]
+    Value* ctx_vector = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_BUILD_EXPR_CTXS, this_ptr, "ctx_vector");
+
+    // Load expr_values_buffer_
+    Value* expr_values_buffer = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUES_BUFFER,
+        this_ptr, "expr_values_buffer");
+
+    // Load expr_value_null_bits_
+    Value* expr_value_null_bits = codegen->CodegenCallFunction(&builder,
+        IRFunction::OLD_HASH_TABLE_GET_EXPR_VALUE_NULL_BITS,
+        this_ptr, "expr_value_null_bits");
+
     for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
       BasicBlock* null_block = BasicBlock::Create(context, "null", fn);
       BasicBlock* not_null_block = BasicBlock::Create(context, "not_null", fn);
       BasicBlock* continue_block = BasicBlock::Create(context, "continue", fn);
 
-      // call GetValue on build_exprs[i]
+      // Generate GetValue() of build_expr_ctxs_[i]
       Function* expr_fn;
       Status status =
           build_expr_ctxs_[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
@@ -635,29 +669,28 @@ Function* OldHashTable::CodegenEquals(LlvmCodeGen* codegen) {
         return NULL;
       }
 
-      Value* ctx_arg = codegen->CastPtrToLlvmPtr(
-          codegen->GetPtrType(ExprContext::LLVM_CLASS_NAME), build_expr_ctxs_[i]);
-      Value* expr_fn_args[] = { ctx_arg, row };
+      // Call GetValue() on build_expr_ctxs_[i]
+      Value* expr_ctx = codegen->CodegenArrayAt(&builder, ctx_vector, i, "expr_ctx");
       CodegenAnyVal result = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
-          build_expr_ctxs_[i]->root()->type(), expr_fn, expr_fn_args, "result");
+          build_expr_ctxs_[i]->root()->type(), expr_fn, {expr_ctx, row}, "result");
       Value* is_null = result.GetIsNull();
 
       // Determine if probe is null (i.e. expr_value_null_bits_[i] == true). In
       // the case where the hash table does not store nulls, this is always false.
       Value* probe_is_null = codegen->false_value();
-      uint8_t* null_byte_loc = &expr_value_null_bits_[i];
       if (stores_nulls_ && finds_nulls_[i]) {
-        Value* llvm_null_byte_loc =
-            codegen->CastPtrToLlvmPtr(codegen->ptr_type(), null_byte_loc);
-        Value* null_byte = builder.CreateLoad(llvm_null_byte_loc);
-        probe_is_null = builder.CreateICmpNE(null_byte,
+        Value* llvm_null_bits_loc = builder.CreateInBoundsGEP(NULL, expr_value_null_bits,
+            codegen->GetIntConstant(TYPE_INT, i), "null_bits_loc");
+        Value* null_bits = builder.CreateLoad(llvm_null_bits_loc, "null_bits");
+        probe_is_null = builder.CreateICmpNE(null_bits,
             codegen->GetIntConstant(TYPE_TINYINT, 0));
       }
 
       // Get llvm value for probe_val from 'expr_values_buffer_'
-      void* loc = expr_values_buffer_ + expr_values_buffer_offsets_[i];
-      Value* probe_val = codegen->CastPtrToLlvmPtr(
-          codegen->GetPtrType(build_expr_ctxs_[i]->root()->type()), loc);
+      Value* probe_val = builder.CreateInBoundsGEP(NULL, expr_values_buffer,
+          codegen->GetIntConstant(TYPE_INT, expr_values_buffer_offsets_[i]), "probe_val");
+      probe_val = builder.CreatePointerCast(
+          probe_val, codegen->GetPtrType(build_expr_ctxs_[i]->root()->type()));
 
       // Branch for GetValue() returning NULL
       builder.CreateCondBr(is_null, null_block, not_null_block);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/56e37166/be/src/exec/old-hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table.h b/be/src/exec/old-hash-table.h
index 3a9b5b2..c5a487d 100644
--- a/be/src/exec/old-hash-table.h
+++ b/be/src/exec/old-hash-table.h
@@ -351,6 +351,14 @@ class OldHashTable {
     Bucket() : node(NULL) { }
   };
 
+  /// Simple wrappers to return various fields in this class. They are done to avoid
+  /// the need to make assumption about the order of declaration of these fields when
+  /// generating the handcrafted IR.
+  uint8_t* IR_ALWAYS_INLINE expr_values_buffer() const;
+  uint8_t* IR_ALWAYS_INLINE expr_value_null_bits() const;
+  ExprContext* const* IR_ALWAYS_INLINE build_expr_ctxs() const;
+  ExprContext* const* IR_ALWAYS_INLINE probe_expr_ctxs() const;
+
   /// Returns the next non-empty bucket and updates idx to be the index of that bucket.
   /// If there are no more buckets, returns NULL and sets idx to -1
   Bucket* NextBucket(int64_t* bucket_idx);