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

[2/5] incubator-impala git commit: IMPALA-6084: Avoid using of global namespace for llvm

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/partitioned-hash-join-builder.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index bf5f891..584b42d 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -46,12 +46,6 @@ static const string PREPARE_FOR_READ_FAILED_ERROR_MSG =
     "the memory limit may help this query to complete successfully.";
 
 using namespace impala;
-using llvm::ConstantInt;
-using llvm::Function;
-using llvm::LLVMContext;
-using llvm::PointerType;
-using llvm::Type;
-using llvm::Value;
 using strings::Substitute;
 
 const char* PhjBuilder::LLVM_CLASS_NAME = "class.impala::PhjBuilder";
@@ -746,16 +740,16 @@ void PhjBuilder::Codegen(LlvmCodeGen* codegen) {
   Status codegen_status;
 
   // Codegen for hashing rows with the builder's hash table context.
-  Function* hash_fn;
+  llvm::Function* hash_fn;
   codegen_status = ht_ctx_->CodegenHashRow(codegen, false, &hash_fn);
-  Function* murmur_hash_fn;
+  llvm::Function* murmur_hash_fn;
   codegen_status.MergeStatus(ht_ctx_->CodegenHashRow(codegen, true, &murmur_hash_fn));
 
   // Codegen for evaluating build rows
-  Function* eval_build_row_fn;
+  llvm::Function* eval_build_row_fn;
   codegen_status.MergeStatus(ht_ctx_->CodegenEvalRow(codegen, true, &eval_build_row_fn));
 
-  Function* insert_filters_fn;
+  llvm::Function* insert_filters_fn;
   codegen_status.MergeStatus(
       CodegenInsertRuntimeFilters(codegen, filter_exprs_, &insert_filters_fn));
 
@@ -786,9 +780,10 @@ string PhjBuilder::DebugString() const {
   return ss.str();
 }
 
-Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen, Function* hash_fn,
-    Function* murmur_hash_fn, Function* eval_row_fn, Function* insert_filters_fn) {
-  Function* process_build_batch_fn =
+Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn,
+    llvm::Function* murmur_hash_fn, llvm::Function* eval_row_fn,
+    llvm::Function* insert_filters_fn) {
+  llvm::Function* process_build_batch_fn =
       codegen->GetFunction(IRFunction::PHJ_PROCESS_BUILD_BATCH, true);
   DCHECK(process_build_batch_fn != NULL);
 
@@ -813,19 +808,20 @@ Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen, Function* hash
   DCHECK_EQ(replaced_constants.stores_tuples, 0);
   DCHECK_EQ(replaced_constants.quadratic_probing, 0);
 
-  Value* is_null_aware_arg = codegen->GetArgument(process_build_batch_fn, 5);
+  llvm::Value* is_null_aware_arg = codegen->GetArgument(process_build_batch_fn, 5);
   is_null_aware_arg->replaceAllUsesWith(
-      ConstantInt::get(Type::getInt1Ty(codegen->context()),
-      join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN));
+      llvm::ConstantInt::get(llvm::Type::getInt1Ty(codegen->context()),
+          join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN));
 
-  Function* process_build_batch_fn_level0 =
+  llvm::Function* process_build_batch_fn_level0 =
       codegen->CloneFunction(process_build_batch_fn);
 
   // Always build runtime filters at level0 (if there are any).
   // Note that the first argument of this function is the return value.
-  Value* build_filter_l0_arg = codegen->GetArgument(process_build_batch_fn_level0, 4);
-  build_filter_l0_arg->replaceAllUsesWith(
-      ConstantInt::get(Type::getInt1Ty(codegen->context()), filter_ctxs_.size() > 0));
+  llvm::Value* build_filter_l0_arg =
+      codegen->GetArgument(process_build_batch_fn_level0, 4);
+  build_filter_l0_arg->replaceAllUsesWith(llvm::ConstantInt::get(
+      llvm::Type::getInt1Ty(codegen->context()), filter_ctxs_.size() > 0));
 
   // process_build_batch_fn_level0 uses CRC hash if available,
   replaced =
@@ -840,9 +836,9 @@ Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen, Function* hash
   // Never build filters after repartitioning, as all rows have already been added to the
   // filters during the level0 build. Note that the first argument of this function is the
   // return value.
-  Value* build_filter_arg = codegen->GetArgument(process_build_batch_fn, 4);
+  llvm::Value* build_filter_arg = codegen->GetArgument(process_build_batch_fn, 4);
   build_filter_arg->replaceAllUsesWith(
-      ConstantInt::get(Type::getInt1Ty(codegen->context()), false));
+      llvm::ConstantInt::get(llvm::Type::getInt1Ty(codegen->context()), false));
 
   // Finalize ProcessBuildBatch functions
   process_build_batch_fn = codegen->FinalizeFunction(process_build_batch_fn);
@@ -867,18 +863,20 @@ Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen, Function* hash
   return Status::OK();
 }
 
-Status PhjBuilder::CodegenInsertBatch(LlvmCodeGen* codegen, Function* hash_fn,
-    Function* murmur_hash_fn, Function* eval_row_fn, TPrefetchMode::type prefetch_mode) {
-  Function* insert_batch_fn = codegen->GetFunction(IRFunction::PHJ_INSERT_BATCH, true);
-  Function* build_equals_fn;
+Status PhjBuilder::CodegenInsertBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn,
+    llvm::Function* murmur_hash_fn, llvm::Function* eval_row_fn,
+    TPrefetchMode::type prefetch_mode) {
+  llvm::Function* insert_batch_fn =
+      codegen->GetFunction(IRFunction::PHJ_INSERT_BATCH, true);
+  llvm::Function* build_equals_fn;
   RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &build_equals_fn));
 
   // Replace the parameter 'prefetch_mode' with constant.
-  Value* prefetch_mode_arg = codegen->GetArgument(insert_batch_fn, 1);
+  llvm::Value* prefetch_mode_arg = codegen->GetArgument(insert_batch_fn, 1);
   DCHECK_GE(prefetch_mode, TPrefetchMode::NONE);
   DCHECK_LE(prefetch_mode, TPrefetchMode::HT_BUCKET);
   prefetch_mode_arg->replaceAllUsesWith(
-      ConstantInt::get(Type::getInt32Ty(codegen->context()), prefetch_mode));
+      llvm::ConstantInt::get(llvm::Type::getInt32Ty(codegen->context()), prefetch_mode));
 
   // Use codegen'd EvalBuildRow() function
   int replaced = codegen->ReplaceCallSites(insert_batch_fn, eval_row_fn, "EvalBuildRow");
@@ -900,7 +898,7 @@ Status PhjBuilder::CodegenInsertBatch(LlvmCodeGen* codegen, Function* hash_fn,
   DCHECK_GE(replaced_constants.stores_tuples, 1);
   DCHECK_GE(replaced_constants.quadratic_probing, 1);
 
-  Function* insert_batch_fn_level0 = codegen->CloneFunction(insert_batch_fn);
+  llvm::Function* insert_batch_fn_level0 = codegen->CloneFunction(insert_batch_fn);
 
   // Use codegen'd hash functions
   replaced = codegen->ReplaceCallSites(insert_batch_fn_level0, hash_fn, "HashRow");
@@ -942,32 +940,32 @@ Status PhjBuilder::CodegenInsertBatch(LlvmCodeGen* codegen, Function* hash_fn,
 //   ret void
 // }
 Status PhjBuilder::CodegenInsertRuntimeFilters(
-    LlvmCodeGen* codegen, const vector<ScalarExpr*>& filter_exprs, Function** fn) {
-  LLVMContext& context = codegen->context();
+    LlvmCodeGen* codegen, const vector<ScalarExpr*>& filter_exprs, llvm::Function** fn) {
+  llvm::LLVMContext& context = codegen->context();
   LlvmBuilder builder(context);
 
   *fn = nullptr;
-  Type* this_type = codegen->GetPtrType(PhjBuilder::LLVM_CLASS_NAME);
-  PointerType* tuple_row_ptr_type = codegen->GetPtrType(TupleRow::LLVM_CLASS_NAME);
+  llvm::Type* this_type = codegen->GetPtrType(PhjBuilder::LLVM_CLASS_NAME);
+  llvm::PointerType* tuple_row_ptr_type = codegen->GetPtrType(TupleRow::LLVM_CLASS_NAME);
   LlvmCodeGen::FnPrototype prototype(
       codegen, "InsertRuntimeFilters", codegen->void_type());
   prototype.AddArgument(LlvmCodeGen::NamedVariable("this", this_type));
   prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
 
-  Value* args[2];
-  Function* insert_runtime_filters_fn = prototype.GeneratePrototype(&builder, args);
-  Value* row_arg = args[1];
+  llvm::Value* args[2];
+  llvm::Function* insert_runtime_filters_fn = prototype.GeneratePrototype(&builder, args);
+  llvm::Value* row_arg = args[1];
 
   int num_filters = filter_exprs.size();
   for (int i = 0; i < num_filters; ++i) {
-    Function* insert_fn;
+    llvm::Function* insert_fn;
     RETURN_IF_ERROR(FilterContext::CodegenInsert(codegen, filter_exprs_[i], &insert_fn));
-    PointerType* filter_context_type =
+    llvm::PointerType* filter_context_type =
         codegen->GetPtrType(FilterContext::LLVM_CLASS_NAME);
-    Value* filter_context_ptr =
+    llvm::Value* filter_context_ptr =
         codegen->CastPtrToLlvmPtr(filter_context_type, &filter_ctxs_[i]);
 
-    Value* insert_args[] = {filter_context_ptr, row_arg};
+    llvm::Value* insert_args[] = {filter_context_ptr, row_arg};
     builder.CreateCall(insert_fn, insert_args);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/partitioned-hash-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-node.cc b/be/src/exec/partitioned-hash-join-node.cc
index 77ed16b..e76a9ba 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -47,14 +47,6 @@ static const string PREPARE_FOR_READ_FAILED_ERROR_MSG =
     "successfully.";
 
 using namespace impala;
-using llvm::BasicBlock;
-using llvm::ConstantInt;
-using llvm::Function;
-using llvm::GlobalValue;
-using llvm::LLVMContext;
-using llvm::PointerType;
-using llvm::Type;
-using llvm::Value;
 using strings::Substitute;
 
 PartitionedHashJoinNode::PartitionedHashJoinNode(
@@ -1301,19 +1293,20 @@ string PartitionedHashJoinNode::NodeDebugString() const {
 //   store i8* null, i8** %dst_tuple_ptr
 //   ret void
 // }
-Status PartitionedHashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen,
-    Function** fn) {
-  Type* tuple_row_type = codegen->GetType(TupleRow::LLVM_CLASS_NAME);
+Status PartitionedHashJoinNode::CodegenCreateOutputRow(
+    LlvmCodeGen* codegen, llvm::Function** fn) {
+  llvm::Type* tuple_row_type = codegen->GetType(TupleRow::LLVM_CLASS_NAME);
   DCHECK(tuple_row_type != NULL);
-  PointerType* tuple_row_ptr_type = PointerType::get(tuple_row_type, 0);
+  llvm::PointerType* tuple_row_ptr_type = llvm::PointerType::get(tuple_row_type, 0);
 
-  Type* this_type = codegen->GetType(BlockingJoinNode::LLVM_CLASS_NAME);
+  llvm::Type* this_type = codegen->GetType(BlockingJoinNode::LLVM_CLASS_NAME);
   DCHECK(this_type != NULL);
-  PointerType* this_ptr_type = PointerType::get(this_type, 0);
+  llvm::PointerType* this_ptr_type = llvm::PointerType::get(this_type, 0);
 
   // TupleRows are really just an array of pointers.  Easier to work with them
   // this way.
-  PointerType* tuple_row_working_type = PointerType::get(codegen->ptr_type(), 0);
+  llvm::PointerType* tuple_row_working_type =
+      llvm::PointerType::get(codegen->ptr_type(), 0);
 
   // Construct function signature to match CreateOutputRow()
   LlvmCodeGen::FnPrototype prototype(codegen, "CreateOutputRow", codegen->void_type());
@@ -1322,33 +1315,37 @@ Status PartitionedHashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen,
   prototype.AddArgument(LlvmCodeGen::NamedVariable("probe_arg", tuple_row_ptr_type));
   prototype.AddArgument(LlvmCodeGen::NamedVariable("build_arg", tuple_row_ptr_type));
 
-  LLVMContext& context = codegen->context();
+  llvm::LLVMContext& context = codegen->context();
   LlvmBuilder builder(context);
-  Value* args[4];
+  llvm::Value* args[4];
   *fn = prototype.GeneratePrototype(&builder, args);
-  Value* out_row_arg = builder.CreateBitCast(args[1], tuple_row_working_type, "out");
-  Value* probe_row_arg = builder.CreateBitCast(args[2], tuple_row_working_type, "probe");
-  Value* build_row_arg = builder.CreateBitCast(args[3], tuple_row_working_type, "build");
+  llvm::Value* out_row_arg =
+      builder.CreateBitCast(args[1], tuple_row_working_type, "out");
+  llvm::Value* probe_row_arg =
+      builder.CreateBitCast(args[2], tuple_row_working_type, "probe");
+  llvm::Value* build_row_arg =
+      builder.CreateBitCast(args[3], tuple_row_working_type, "build");
 
   int num_probe_tuples = child(0)->row_desc()->tuple_descriptors().size();
   int num_build_tuples = child(1)->row_desc()->tuple_descriptors().size();
 
   // Copy probe row
   codegen->CodegenMemcpy(&builder, out_row_arg, probe_row_arg, probe_tuple_row_size_);
-  Value* build_row_idx[] = {codegen->GetIntConstant(TYPE_INT, num_probe_tuples)};
-  Value* build_row_dst =
+  llvm::Value* build_row_idx[] = {codegen->GetIntConstant(TYPE_INT, num_probe_tuples)};
+  llvm::Value* build_row_dst =
       builder.CreateInBoundsGEP(out_row_arg, build_row_idx, "build_dst_ptr");
 
   // Copy build row.
-  BasicBlock* build_not_null_block = BasicBlock::Create(context, "build_not_null", *fn);
-  BasicBlock* build_null_block = NULL;
+  llvm::BasicBlock* build_not_null_block =
+      llvm::BasicBlock::Create(context, "build_not_null", *fn);
+  llvm::BasicBlock* build_null_block = NULL;
 
   if (join_op_ == TJoinOp::LEFT_ANTI_JOIN || join_op_ == TJoinOp::LEFT_OUTER_JOIN ||
       join_op_ == TJoinOp::FULL_OUTER_JOIN ||
       join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
     // build tuple can be null
-    build_null_block = BasicBlock::Create(context, "build_null", *fn);
-    Value* is_build_null = builder.CreateIsNull(build_row_arg, "is_build_null");
+    build_null_block = llvm::BasicBlock::Create(context, "build_null", *fn);
+    llvm::Value* is_build_null = builder.CreateIsNull(build_row_arg, "is_build_null");
     builder.CreateCondBr(is_build_null, build_null_block, build_not_null_block);
 
     // Set tuple build ptrs to NULL
@@ -1356,8 +1353,10 @@ Status PartitionedHashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen,
     // to work.
     builder.SetInsertPoint(build_null_block);
     for (int i = 0; i < num_build_tuples; ++i) {
-      Value* array_idx[] = {codegen->GetIntConstant(TYPE_INT, i + num_probe_tuples)};
-      Value* dst = builder.CreateInBoundsGEP(out_row_arg, array_idx, "dst_tuple_ptr");
+      llvm::Value* array_idx[] = {
+          codegen->GetIntConstant(TYPE_INT, i + num_probe_tuples)};
+      llvm::Value* dst =
+          builder.CreateInBoundsGEP(out_row_arg, array_idx, "dst_tuple_ptr");
       builder.CreateStore(codegen->null_ptr_value(), dst);
     }
     builder.CreateRetVoid();
@@ -1382,8 +1381,8 @@ Status PartitionedHashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen,
 Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
     LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
   // Codegen for hashing rows
-  Function* hash_fn;
-  Function* murmur_hash_fn;
+  llvm::Function* hash_fn;
+  llvm::Function* murmur_hash_fn;
   RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, false, &hash_fn));
   RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, true, &murmur_hash_fn));
 
@@ -1420,41 +1419,41 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
     default:
       DCHECK(false);
   }
-  Function* process_probe_batch_fn = codegen->GetFunction(ir_fn, true);
+  llvm::Function* process_probe_batch_fn = codegen->GetFunction(ir_fn, true);
   DCHECK(process_probe_batch_fn != NULL);
   process_probe_batch_fn->setName("ProcessProbeBatch");
 
   // Verifies that ProcessProbeBatch() has weak_odr linkage so it's not discarded even
   // if it's not referenced. See http://llvm.org/docs/LangRef.html#linkage-types
-  DCHECK(process_probe_batch_fn->getLinkage() == GlobalValue::WeakODRLinkage)
+  DCHECK(process_probe_batch_fn->getLinkage() == llvm::GlobalValue::WeakODRLinkage)
       << LlvmCodeGen::Print(process_probe_batch_fn);
 
   // Replace the parameter 'prefetch_mode' with constant.
-  Value* prefetch_mode_arg = codegen->GetArgument(process_probe_batch_fn, 1);
+  llvm::Value* prefetch_mode_arg = codegen->GetArgument(process_probe_batch_fn, 1);
   DCHECK_GE(prefetch_mode, TPrefetchMode::NONE);
   DCHECK_LE(prefetch_mode, TPrefetchMode::HT_BUCKET);
   prefetch_mode_arg->replaceAllUsesWith(
-      ConstantInt::get(Type::getInt32Ty(codegen->context()), prefetch_mode));
+      llvm::ConstantInt::get(llvm::Type::getInt32Ty(codegen->context()), prefetch_mode));
 
   // Codegen HashTable::Equals
-  Function* probe_equals_fn;
+  llvm::Function* probe_equals_fn;
   RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, false, &probe_equals_fn));
 
   // Codegen for evaluating probe rows
-  Function* eval_row_fn;
+  llvm::Function* eval_row_fn;
   RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_row_fn));
 
   // Codegen CreateOutputRow
-  Function* create_output_row_fn;
+  llvm::Function* create_output_row_fn;
   RETURN_IF_ERROR(CodegenCreateOutputRow(codegen, &create_output_row_fn));
 
   // Codegen evaluating other join conjuncts
-  Function* eval_other_conjuncts_fn;
+  llvm::Function* eval_other_conjuncts_fn;
   RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, other_join_conjuncts_,
       &eval_other_conjuncts_fn, "EvalOtherConjuncts"));
 
   // Codegen evaluating conjuncts
-  Function* eval_conjuncts_fn;
+  llvm::Function* eval_conjuncts_fn;
   RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, conjuncts_,
       &eval_conjuncts_fn));
 
@@ -1512,7 +1511,7 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
   DCHECK_GE(replaced_constants.stores_tuples, 1);
   DCHECK_GE(replaced_constants.quadratic_probing, 1);
 
-  Function* process_probe_batch_fn_level0 =
+  llvm::Function* process_probe_batch_fn_level0 =
       codegen->CloneFunction(process_probe_batch_fn);
 
   // process_probe_batch_fn_level0 uses CRC hash if available,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/select-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/select-node.cc b/be/src/exec/select-node.cc
index b82b3e3..0f0683b 100644
--- a/be/src/exec/select-node.cc
+++ b/be/src/exec/select-node.cc
@@ -27,8 +27,6 @@
 
 #include "common/names.h"
 
-using llvm::Function;
-
 namespace impala {
 
 SelectNode::SelectNode(
@@ -58,10 +56,11 @@ void SelectNode::Codegen(RuntimeState* state) {
 Status SelectNode::CodegenCopyRows(RuntimeState* state) {
   LlvmCodeGen* codegen = state->codegen();
   DCHECK(codegen != nullptr);
-  Function* copy_rows_fn = codegen->GetFunction(IRFunction::SELECT_NODE_COPY_ROWS, true);
+  llvm::Function* copy_rows_fn =
+      codegen->GetFunction(IRFunction::SELECT_NODE_COPY_ROWS, true);
   DCHECK(copy_rows_fn != nullptr);
 
-  Function* eval_conjuncts_fn;
+  llvm::Function* eval_conjuncts_fn;
   RETURN_IF_ERROR(
       ExecNode::CodegenEvalConjuncts(codegen, conjuncts_, &eval_conjuncts_fn));
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/text-converter.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/text-converter.cc b/be/src/exec/text-converter.cc
index 3bb65f2..ddd0d6e 100644
--- a/be/src/exec/text-converter.cc
+++ b/be/src/exec/text-converter.cc
@@ -31,7 +31,6 @@
 #include "common/names.h"
 
 using namespace impala;
-using namespace llvm;
 
 TextConverter::TextConverter(char escape_char, const string& null_col_val,
     bool check_null, bool strict_mode)
@@ -106,7 +105,7 @@ void TextConverter::UnescapeString(const char* src, char* dest, int* len,
 // TODO: convert this function to use cross-compilation + constant substitution in whole
 // or part. It is currently too complex and doesn't implement the full functionality.
 Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
-    TupleDescriptor* tuple_desc, SlotDescriptor* slot_desc, Function** fn,
+    TupleDescriptor* tuple_desc, SlotDescriptor* slot_desc, llvm::Function** fn,
     const char* null_col_val, int len, bool check_null, bool strict_mode) {
   DCHECK(fn != nullptr);
   if (slot_desc->type().type == TYPE_CHAR) {
@@ -117,7 +116,7 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
 
   // Codegen is_null_string
   bool is_default_null = (len == 2 && null_col_val[0] == '\\' && null_col_val[1] == 'N');
-  Function* is_null_string_fn;
+  llvm::Function* is_null_string_fn;
   if (is_default_null) {
     is_null_string_fn = codegen->GetFunction(IRFunction::IS_NULL_STRING, false);
   } else {
@@ -126,12 +125,12 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
 
   DCHECK(is_null_string_fn != NULL);
 
-  StructType* tuple_type = tuple_desc->GetLlvmStruct(codegen);
+  llvm::StructType* tuple_type = tuple_desc->GetLlvmStruct(codegen);
   if (tuple_type == NULL) {
     return Status("TextConverter::CodegenWriteSlot(): Failed to generate "
         "tuple type");
   }
-  PointerType* tuple_ptr_type = tuple_type->getPointerTo();
+  llvm::PointerType* tuple_ptr_type = tuple_type->getPointerTo();
 
   LlvmCodeGen::FnPrototype prototype(
       codegen, "WriteSlot", codegen->GetType(TYPE_BOOLEAN));
@@ -140,27 +139,29 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
   prototype.AddArgument(LlvmCodeGen::NamedVariable("len", codegen->GetType(TYPE_INT)));
 
   LlvmBuilder builder(codegen->context());
-  Value* args[3];
+  llvm::Value* args[3];
   *fn = prototype.GeneratePrototype(&builder, &args[0]);
 
-  BasicBlock* set_null_block, *parse_slot_block, *check_zero_block = NULL;
+  llvm::BasicBlock *set_null_block, *parse_slot_block, *check_zero_block = NULL;
   codegen->CreateIfElseBlocks(*fn, "set_null", "parse_slot",
       &set_null_block, &parse_slot_block);
 
   if (!slot_desc->type().IsVarLenStringType()) {
-    check_zero_block = BasicBlock::Create(codegen->context(), "check_zero", *fn);
+    check_zero_block = llvm::BasicBlock::Create(codegen->context(), "check_zero", *fn);
   }
 
   // Check if the data matches the configured NULL string.
-  Value* is_null;
+  llvm::Value* is_null;
   if (check_null) {
     if (is_default_null) {
-      is_null = builder.CreateCall(is_null_string_fn,
-          ArrayRef<Value*>({args[1], args[2]}));
+      is_null = builder.CreateCall(
+          is_null_string_fn, llvm::ArrayRef<llvm::Value*>({args[1], args[2]}));
     } else {
-      is_null = builder.CreateCall(is_null_string_fn, ArrayRef<Value*>({args[1], args[2],
-          codegen->CastPtrToLlvmPtr(codegen->ptr_type(), const_cast<char*>(null_col_val)),
-          codegen->GetIntConstant(TYPE_INT, len)}));
+      is_null = builder.CreateCall(is_null_string_fn,
+          llvm::ArrayRef<llvm::Value*>(
+              {args[1], args[2], codegen->CastPtrToLlvmPtr(codegen->ptr_type(),
+                                     const_cast<char*>(null_col_val)),
+                  codegen->GetIntConstant(TYPE_INT, len)}));
     }
   } else {
     // Constant FALSE as branch condition. We rely on later optimization passes
@@ -173,29 +174,30 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
   if (!slot_desc->type().IsVarLenStringType()) {
     builder.SetInsertPoint(check_zero_block);
     // If len == 0 and it is not a string col, set slot to NULL
-    Value* null_len = builder.CreateICmpEQ(
-        args[2], codegen->GetIntConstant(TYPE_INT, 0));
+    llvm::Value* null_len =
+        builder.CreateICmpEQ(args[2], codegen->GetIntConstant(TYPE_INT, 0));
     builder.CreateCondBr(null_len, set_null_block, parse_slot_block);
   }
 
   // Codegen parse slot block
   builder.SetInsertPoint(parse_slot_block);
-  Value* slot = builder.CreateStructGEP(NULL, args[0], slot_desc->llvm_field_idx(),
-      "slot");
+  llvm::Value* slot =
+      builder.CreateStructGEP(NULL, args[0], slot_desc->llvm_field_idx(), "slot");
 
   if (slot_desc->type().IsVarLenStringType()) {
-    Value* ptr = builder.CreateStructGEP(NULL, slot, 0, "string_ptr");
-    Value* len = builder.CreateStructGEP(NULL, slot, 1, "string_len");
+    llvm::Value* ptr = builder.CreateStructGEP(NULL, slot, 0, "string_ptr");
+    llvm::Value* len = builder.CreateStructGEP(NULL, slot, 1, "string_len");
 
     builder.CreateStore(args[1], ptr);
     // TODO codegen memory allocation for CHAR
     DCHECK(slot_desc->type().type != TYPE_CHAR);
     if (slot_desc->type().type == TYPE_VARCHAR) {
       // determine if we need to truncate the string
-      Value* maxlen = codegen->GetIntConstant(TYPE_INT, slot_desc->type().len);
-      Value* len_lt_maxlen = builder.CreateICmpSLT(args[2], maxlen, "len_lt_maxlen");
-      Value* minlen = builder.CreateSelect(len_lt_maxlen, args[2], maxlen,
-                                           "select_min_len");
+      llvm::Value* maxlen = codegen->GetIntConstant(TYPE_INT, slot_desc->type().len);
+      llvm::Value* len_lt_maxlen =
+          builder.CreateICmpSLT(args[2], maxlen, "len_lt_maxlen");
+      llvm::Value* minlen =
+          builder.CreateSelect(len_lt_maxlen, args[2], maxlen, "select_min_len");
       builder.CreateStore(minlen, len);
     } else {
       builder.CreateStore(args[2], len);
@@ -203,7 +205,7 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
     builder.CreateRet(codegen->true_value());
   } else {
     IRFunction::Type parse_fn_enum;
-    Function* parse_fn = NULL;
+    llvm::Function* parse_fn = NULL;
     switch (slot_desc->type().type) {
       case TYPE_BOOLEAN:
         parse_fn_enum = IRFunction::STRING_TO_BOOL;
@@ -255,13 +257,13 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
     DCHECK(parse_fn != NULL);
 
     // Set up trying to parse the string to the slot type
-    BasicBlock* parse_success_block, *parse_failed_block;
+    llvm::BasicBlock *parse_success_block, *parse_failed_block;
     codegen->CreateIfElseBlocks(*fn, "parse_success", "parse_fail",
         &parse_success_block, &parse_failed_block);
     LlvmCodeGen::NamedVariable parse_result("parse_result", codegen->GetType(TYPE_INT));
-    Value* parse_result_ptr = codegen->CreateEntryBlockAlloca(*fn, parse_result);
+    llvm::Value* parse_result_ptr = codegen->CreateEntryBlockAlloca(*fn, parse_result);
 
-    CallInst* parse_return;
+    llvm::CallInst* parse_return;
     // Call Impala's StringTo* function
     // Function implementations in exec/hdfs-scanner-ir.cc
     if (slot_desc->type().type == TYPE_DECIMAL) {
@@ -278,17 +280,19 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
     } else {
       parse_return = builder.CreateCall(parse_fn, {args[1], args[2], parse_result_ptr});
     }
-    Value* parse_result_val = builder.CreateLoad(parse_result_ptr, "parse_result");
-    Value* failed_value = codegen->GetIntConstant(TYPE_INT, StringParser::PARSE_FAILURE);
+    llvm::Value* parse_result_val = builder.CreateLoad(parse_result_ptr, "parse_result");
+    llvm::Value* failed_value =
+        codegen->GetIntConstant(TYPE_INT, StringParser::PARSE_FAILURE);
 
     // Check for parse error.
-    Value* parse_failed = builder.CreateICmpEQ(parse_result_val, failed_value, "failed");
+    llvm::Value* parse_failed =
+        builder.CreateICmpEQ(parse_result_val, failed_value, "failed");
     if (strict_mode) {
       // In strict_mode, also check if parse_result is PARSE_OVERFLOW.
-      Value* overflow_value = codegen->GetIntConstant(TYPE_INT,
-          StringParser::PARSE_OVERFLOW);
-      Value* parse_overflow = builder.CreateICmpEQ(parse_result_val, overflow_value,
-          "overflowed");
+      llvm::Value* overflow_value =
+          codegen->GetIntConstant(TYPE_INT, StringParser::PARSE_OVERFLOW);
+      llvm::Value* parse_overflow =
+          builder.CreateICmpEQ(parse_result_val, overflow_value, "overflowed");
       parse_failed = builder.CreateOr(parse_failed, parse_overflow, "failed_or");
     }
     builder.CreateCondBr(parse_failed, parse_failed_block, parse_success_block);
@@ -299,8 +303,8 @@ Status TextConverter::CodegenWriteSlot(LlvmCodeGen* codegen,
     if (slot_desc->type().type == TYPE_DECIMAL) {
       // For Decimal values, the return type generated by Clang is struct type rather than
       // integer so casting is necessary
-      Value* cast_slot = builder.CreateBitCast(slot,
-          parse_return->getType()->getPointerTo());
+      llvm::Value* cast_slot =
+          builder.CreateBitCast(slot, parse_return->getType()->getPointerTo());
       builder.CreateStore(parse_return, cast_slot);
     } else if (slot_desc->type().type != TYPE_TIMESTAMP) {
       builder.CreateStore(parse_return, slot);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/topn-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.cc b/be/src/exec/topn-node.cc
index 6b4946f..31bf58b 100644
--- a/be/src/exec/topn-node.cc
+++ b/be/src/exec/topn-node.cc
@@ -38,7 +38,6 @@
 
 using std::priority_queue;
 using namespace impala;
-using namespace llvm;
 
 TopNNode::TopNNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
   : ExecNode(pool, tnode, descs),
@@ -97,15 +96,15 @@ void TopNNode::Codegen(RuntimeState* state) {
   // TODO: inline tuple_row_less_than_->Compare()
   Status codegen_status = tuple_row_less_than_->Codegen(state);
   if (codegen_status.ok()) {
-    Function* insert_batch_fn =
+    llvm::Function* insert_batch_fn =
         codegen->GetFunction(IRFunction::TOPN_NODE_INSERT_BATCH, true);
     DCHECK(insert_batch_fn != NULL);
 
     // Generate two MaterializeExprs() functions, one using tuple_pool_ and
     // one with no pool.
     DCHECK(output_tuple_desc_ != NULL);
-    Function* materialize_exprs_tuple_pool_fn;
-    Function* materialize_exprs_no_pool_fn;
+    llvm::Function* materialize_exprs_tuple_pool_fn;
+    llvm::Function* materialize_exprs_no_pool_fn;
 
     codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
         *output_tuple_desc_, output_tuple_exprs_,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exec/union-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/union-node.cc b/be/src/exec/union-node.cc
index f2acc7a..a857ed1 100644
--- a/be/src/exec/union-node.cc
+++ b/be/src/exec/union-node.cc
@@ -28,7 +28,6 @@
 
 #include "common/names.h"
 
-using namespace llvm;
 using namespace impala;
 
 UnionNode::UnionNode(ObjectPool* pool, const TPlanNode& tnode,
@@ -118,7 +117,7 @@ void UnionNode::Codegen(RuntimeState* state) {
 
     // Get a copy of the function. This function will be modified and added to the
     // vector of functions.
-    Function* union_materialize_batch_fn =
+    llvm::Function* union_materialize_batch_fn =
         codegen->GetFunction(IRFunction::UNION_MATERIALIZE_BATCH, true);
     DCHECK(union_materialize_batch_fn != nullptr);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/agg-fn-evaluator.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/agg-fn-evaluator.cc b/be/src/exprs/agg-fn-evaluator.cc
index 1af76f4..3ba8f37 100644
--- a/be/src/exprs/agg-fn-evaluator.cc
+++ b/be/src/exprs/agg-fn-evaluator.cc
@@ -40,7 +40,6 @@
 
 using namespace impala;
 using namespace impala_udf;
-using namespace llvm;
 using std::move;
 
 // typedef for builtin aggregate functions. Unfortunately, these type defs don't

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/agg-fn.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/agg-fn.cc b/be/src/exprs/agg-fn.cc
index 68ea7bf..dee3944 100644
--- a/be/src/exprs/agg-fn.cc
+++ b/be/src/exprs/agg-fn.cc
@@ -26,7 +26,6 @@
 #include "common/names.h"
 
 using namespace impala_udf;
-using namespace llvm;
 
 namespace impala {
 
@@ -148,7 +147,8 @@ FunctionContext::TypeDesc AggFn::GetOutputTypeDesc() const {
   return AnyValUtil::ColumnTypeToTypeDesc(output_slot_desc_.type());
 }
 
-Status AggFn::CodegenUpdateOrMergeFunction(LlvmCodeGen* codegen, Function** uda_fn) {
+Status AggFn::CodegenUpdateOrMergeFunction(
+    LlvmCodeGen* codegen, llvm::Function** uda_fn) {
   const string& symbol =
       is_merge_ ? fn_.aggregate_fn.merge_fn_symbol : fn_.aggregate_fn.update_fn_symbol;
   vector<ColumnType> fn_arg_types;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/case-expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/case-expr.cc b/be/src/exprs/case-expr.cc
index 5a3d1fb..62460d6 100644
--- a/be/src/exprs/case-expr.cc
+++ b/be/src/exprs/case-expr.cc
@@ -28,8 +28,6 @@
 
 #include "common/names.h"
 
-using namespace llvm;
-
 namespace impala {
 
 struct CaseExprState {
@@ -177,42 +175,42 @@ string CaseExpr::DebugString() const {
 //                                   %"class.impala::TupleRow"* %row)
 //   ret i16 %else_val
 // }
-Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
+Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != nullptr) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
   const int num_children = GetNumChildren();
-  Function* child_fns[num_children];
+  llvm::Function* child_fns[num_children];
   for (int i = 0; i < num_children; ++i) {
     RETURN_IF_ERROR(GetChild(i)->GetCodegendComputeFn(codegen, &child_fns[i]));
   }
 
-  LLVMContext& context = codegen->context();
+  llvm::LLVMContext& context = codegen->context();
   LlvmBuilder builder(context);
 
-  Value* args[2];
-  Function* function = CreateIrFunctionPrototype("CaseExpr", codegen, &args);
-  BasicBlock* eval_case_expr_block = nullptr;
+  llvm::Value* args[2];
+  llvm::Function* function = CreateIrFunctionPrototype("CaseExpr", codegen, &args);
+  llvm::BasicBlock* eval_case_expr_block = nullptr;
 
   // This is the block immediately after the when/then exprs. It will either point to a
   // block which returns the else expr, or returns NULL if no else expr is specified.
-  BasicBlock* default_value_block = BasicBlock::Create(
+  llvm::BasicBlock* default_value_block = llvm::BasicBlock::Create(
       context, has_else_expr() ? "return_else_expr" : "return_null", function);
 
   // If there is a case expression, create a block to evaluate it.
   CodegenAnyVal case_val;
-  BasicBlock* eval_first_when_expr_block = BasicBlock::Create(
+  llvm::BasicBlock* eval_first_when_expr_block = llvm::BasicBlock::Create(
       context, "eval_first_when_expr", function, default_value_block);
-  BasicBlock* current_when_expr_block = eval_first_when_expr_block;
+  llvm::BasicBlock* current_when_expr_block = eval_first_when_expr_block;
   if (has_case_expr()) {
     // Need at least case, when and then expr, and optionally an else expr
     DCHECK_GE(num_children, has_else_expr() ? 4 : 3);
     // If there is a case expr, create block eval_case_expr to evaluate the
     // case expr. Place this block before eval_first_when_expr_block
-    eval_case_expr_block = BasicBlock::Create(context, "eval_case_expr",
-        function, eval_first_when_expr_block);
+    eval_case_expr_block = llvm::BasicBlock::Create(
+        context, "eval_case_expr", function, eval_first_when_expr_block);
     builder.SetInsertPoint(eval_case_expr_block);
     case_val = CodegenAnyVal::CreateCallWrapped(
         codegen, &builder, children()[0]->type(), child_fns[0], args, "case_val");
@@ -228,18 +226,18 @@ Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
   // pair. Both when and then subexpressions are single children. If there is a case expr
   // start loop at index 1. (case expr is GetChild(0) and has already be evaluated.
   for (int i = has_case_expr() ? 1 : 0; i < loop_end; i += 2) {
-    BasicBlock* check_when_expr_block = BasicBlock::Create(
+    llvm::BasicBlock* check_when_expr_block = llvm::BasicBlock::Create(
         context, "check_when_expr_block", function, default_value_block);
-    BasicBlock* return_then_expr_block =
-        BasicBlock::Create(context, "return_then_expr", function, default_value_block);
+    llvm::BasicBlock* return_then_expr_block = llvm::BasicBlock::Create(
+        context, "return_then_expr", function, default_value_block);
 
     // continue_or_exit_block either points to the next eval_next_when_expr block,
     // or points to the defaut_value_block if there are no more when/then expressions.
-    BasicBlock* continue_or_exit_block = nullptr;
+    llvm::BasicBlock* continue_or_exit_block = nullptr;
     if (i == last_loop_iter) {
       continue_or_exit_block = default_value_block;
     } else {
-      continue_or_exit_block = BasicBlock::Create(
+      continue_or_exit_block = llvm::BasicBlock::Create(
           context, "eval_next_when_expr", function, default_value_block);
     }
 
@@ -254,7 +252,7 @@ Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
     builder.SetInsertPoint(check_when_expr_block);
     if (has_case_expr()) {
       // Compare for equality
-      Value* is_equal = case_val.Eq(&when_val);
+      llvm::Value* is_equal = case_val.Eq(&when_val);
       builder.CreateCondBr(is_equal, return_then_expr_block, continue_or_exit_block);
     } else {
       builder.CreateCondBr(
@@ -264,8 +262,8 @@ Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
     builder.SetInsertPoint(return_then_expr_block);
 
     // Eval and return then value
-    Value* then_val = CodegenAnyVal::CreateCall(
-        codegen, &builder, child_fns[i+1], args, "then_val");
+    llvm::Value* then_val =
+        CodegenAnyVal::CreateCall(codegen, &builder, child_fns[i + 1], args, "then_val");
     builder.CreateRet(then_val);
 
     current_when_expr_block = continue_or_exit_block;
@@ -273,7 +271,7 @@ Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
 
   builder.SetInsertPoint(default_value_block);
   if (has_else_expr()) {
-    Value* else_val = CodegenAnyVal::CreateCall(
+    llvm::Value* else_val = CodegenAnyVal::CreateCall(
         codegen, &builder, child_fns[num_children - 1], args, "else_val");
     builder.CreateRet(else_val);
   } else {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/compound-predicates.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/compound-predicates.cc b/be/src/exprs/compound-predicates.cc
index 2474995..4d89cfa 100644
--- a/be/src/exprs/compound-predicates.cc
+++ b/be/src/exprs/compound-predicates.cc
@@ -25,7 +25,6 @@
 #include "common/names.h"
 
 using namespace impala;
-using namespace llvm;
 
 // (<> && false) is false, (true && NULL) is NULL
 BooleanVal AndPredicate::GetBooleanVal(ScalarExprEvaluator* eval,
@@ -121,36 +120,41 @@ string OrPredicate::DebugString() const {
 //   ret i16 %7
 // }
 Status CompoundPredicate::CodegenComputeFn(
-    bool and_fn, LlvmCodeGen* codegen, Function** fn) {
+    bool and_fn, LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
   DCHECK_EQ(GetNumChildren(), 2);
-  Function* lhs_function;
+  llvm::Function* lhs_function;
   RETURN_IF_ERROR(children()[0]->GetCodegendComputeFn(codegen, &lhs_function));
-  Function* rhs_function;
+  llvm::Function* rhs_function;
   RETURN_IF_ERROR(children()[1]->GetCodegendComputeFn(codegen, &rhs_function));
 
-  LLVMContext& context = codegen->context();
+  llvm::LLVMContext& context = codegen->context();
   LlvmBuilder builder(context);
-  Value* args[2];
-  Function* function = CreateIrFunctionPrototype("CompoundPredicate", codegen, &args);
+  llvm::Value* args[2];
+  llvm::Function* function =
+      CreateIrFunctionPrototype("CompoundPredicate", codegen, &args);
 
-  BasicBlock* entry_block = BasicBlock::Create(context, "entry", function);
+  llvm::BasicBlock* entry_block = llvm::BasicBlock::Create(context, "entry", function);
   builder.SetInsertPoint(entry_block);
 
   // Control blocks for aggregating results
-  BasicBlock* lhs_null_block = BasicBlock::Create(context, "lhs_null", function);
-  BasicBlock* lhs_not_null_block = BasicBlock::Create(context, "lhs_not_null", function);
-  BasicBlock* lhs_null_rhs_not_null_block =
-      BasicBlock::Create(context, "lhs_null_rhs_not_null", function);
-  BasicBlock* lhs_not_null_rhs_null_block =
-      BasicBlock::Create(context, "lhs_not_null_rhs_null", function);
-  BasicBlock* null_block = BasicBlock::Create(context, "null_block", function);
-  BasicBlock* not_null_block = BasicBlock::Create(context, "not_null_block", function);
-  BasicBlock* ret_block = BasicBlock::Create(context, "ret", function);
+  llvm::BasicBlock* lhs_null_block =
+      llvm::BasicBlock::Create(context, "lhs_null", function);
+  llvm::BasicBlock* lhs_not_null_block =
+      llvm::BasicBlock::Create(context, "lhs_not_null", function);
+  llvm::BasicBlock* lhs_null_rhs_not_null_block =
+      llvm::BasicBlock::Create(context, "lhs_null_rhs_not_null", function);
+  llvm::BasicBlock* lhs_not_null_rhs_null_block =
+      llvm::BasicBlock::Create(context, "lhs_not_null_rhs_null", function);
+  llvm::BasicBlock* null_block =
+      llvm::BasicBlock::Create(context, "null_block", function);
+  llvm::BasicBlock* not_null_block =
+      llvm::BasicBlock::Create(context, "not_null_block", function);
+  llvm::BasicBlock* ret_block = llvm::BasicBlock::Create(context, "ret", function);
 
   // Call lhs
   CodegenAnyVal lhs_result = CodegenAnyVal::CreateCallWrapped(
@@ -159,13 +163,13 @@ Status CompoundPredicate::CodegenComputeFn(
   CodegenAnyVal rhs_result = CodegenAnyVal::CreateCallWrapped(
       codegen, &builder, TYPE_BOOLEAN, rhs_function, args, "rhs_call");
 
-  Value* lhs_is_null = lhs_result.GetIsNull();
-  Value* rhs_is_null = rhs_result.GetIsNull();
-  Value* lhs_value = lhs_result.GetVal();
-  Value* rhs_value = rhs_result.GetVal();
+  llvm::Value* lhs_is_null = lhs_result.GetIsNull();
+  llvm::Value* rhs_is_null = rhs_result.GetIsNull();
+  llvm::Value* lhs_value = lhs_result.GetVal();
+  llvm::Value* rhs_value = rhs_result.GetVal();
 
   // Apply predicate
-  Value* compare = NULL;
+  llvm::Value* compare = NULL;
   if (and_fn) {
     compare = builder.CreateAnd(lhs_value, rhs_value, "tmp_and");
   } else {
@@ -209,7 +213,7 @@ Status CompoundPredicate::CodegenComputeFn(
 
   // not-NULL block
   builder.SetInsertPoint(not_null_block);
-  PHINode* not_null_phi = builder.CreatePHI(codegen->GetType(TYPE_BOOLEAN), 3);
+  llvm::PHINode* not_null_phi = builder.CreatePHI(codegen->GetType(TYPE_BOOLEAN), 3);
   if (and_fn) {
     not_null_phi->addIncoming(codegen->false_value(), lhs_null_rhs_not_null_block);
     not_null_phi->addIncoming(codegen->false_value(), lhs_not_null_rhs_null_block);
@@ -223,11 +227,11 @@ Status CompoundPredicate::CodegenComputeFn(
 
   // Ret/merge block
   builder.SetInsertPoint(ret_block);
-  PHINode* is_null_phi = builder.CreatePHI(codegen->boolean_type(), 2, "is_null");
+  llvm::PHINode* is_null_phi = builder.CreatePHI(codegen->boolean_type(), 2, "is_null");
   is_null_phi->addIncoming(codegen->true_value(), null_block);
   is_null_phi->addIncoming(codegen->false_value(), not_null_block);
 
-  PHINode* val_phi = builder.CreatePHI(codegen->boolean_type(), 2, "val");
+  llvm::PHINode* val_phi = builder.CreatePHI(codegen->boolean_type(), 2, "val");
   val_phi->addIncoming(codegen->false_value(), null_block);
   val_phi->addIncoming(not_null_phi, not_null_block);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/expr-codegen-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-codegen-test.cc b/be/src/exprs/expr-codegen-test.cc
index 4b13073..781e457 100644
--- a/be/src/exprs/expr-codegen-test.cc
+++ b/be/src/exprs/expr-codegen-test.cc
@@ -73,8 +73,6 @@ DecimalVal TestGetFnAttrs(
 
 #include "common/names.h"
 
-using namespace llvm;
-
 namespace impala {
 
 const char* TEST_GET_FN_ATTR_SYMBOL =
@@ -317,7 +315,7 @@ TEST_F(ExprCodegenTest, TestInlineConstFnAttrs) {
   test_udf_file << getenv("IMPALA_HOME") << "/be/build/latest/exprs/expr-codegen-test.ll";
   scoped_ptr<LlvmCodeGen> codegen;
   ASSERT_OK(CreateFromFile(test_udf_file.str(), &codegen));
-  Function* fn = codegen->GetFunction(TEST_GET_FN_ATTR_SYMBOL, false);
+  llvm::Function* fn = codegen->GetFunction(TEST_GET_FN_ATTR_SYMBOL, false);
   ASSERT_TRUE(fn != NULL);
 
   // Function verification should fail because we haven't inlined GetTypeAttr() calls

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index b509da6..d43f424 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -77,7 +77,6 @@ using boost::posix_time::to_tm;
 using std::numeric_limits;
 using namespace Apache::Hadoop::Hive;
 using namespace impala;
-using namespace llvm;
 
 namespace impala {
 ImpaladQueryExecutor* executor_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/literal.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/literal.cc b/be/src/exprs/literal.cc
index 20f4ba0..92bc0c1 100644
--- a/be/src/exprs/literal.cc
+++ b/be/src/exprs/literal.cc
@@ -30,7 +30,6 @@
 
 #include "common/names.h"
 
-using namespace llvm;
 using namespace impala_udf;
 
 namespace impala {
@@ -358,9 +357,10 @@ Status Literal::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   }
 
   DCHECK_EQ(GetNumChildren(), 0);
-  Value* args[2];
+  llvm::Value* args[2];
   *fn = CreateIrFunctionPrototype("Literal", codegen, &args);
-  BasicBlock* entry_block = BasicBlock::Create(codegen->context(), "entry", *fn);
+  llvm::BasicBlock* entry_block =
+      llvm::BasicBlock::Create(codegen->context(), "entry", *fn);
   LlvmBuilder builder(entry_block);
 
   CodegenAnyVal v = CodegenAnyVal::GetNonNullVal(codegen, &builder, type_);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/null-literal.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/null-literal.cc b/be/src/exprs/null-literal.cc
index 65064e6..d2d4590 100644
--- a/be/src/exprs/null-literal.cc
+++ b/be/src/exprs/null-literal.cc
@@ -26,7 +26,6 @@
 #include "common/names.h"
 
 using namespace impala_udf;
-using namespace llvm;
 
 namespace impala {
 
@@ -110,12 +109,13 @@ Status NullLiteral::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function**
   }
 
   DCHECK_EQ(GetNumChildren(), 0);
-  Value* args[2];
+  llvm::Value* args[2];
   *fn = CreateIrFunctionPrototype("NullLiteral", codegen, &args);
-  BasicBlock* entry_block = BasicBlock::Create(codegen->context(), "entry", *fn);
+  llvm::BasicBlock* entry_block =
+      llvm::BasicBlock::Create(codegen->context(), "entry", *fn);
   LlvmBuilder builder(entry_block);
 
-  Value* v = CodegenAnyVal::GetNullVal(codegen, type());
+  llvm::Value* v = CodegenAnyVal::GetNullVal(codegen, type());
   builder.CreateRet(v);
   *fn = codegen->FinalizeFunction(*fn);
   if (UNLIKELY(*fn == nullptr)) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/scalar-expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-expr.cc b/be/src/exprs/scalar-expr.cc
index 04aaa6a..035dad5 100644
--- a/be/src/exprs/scalar-expr.cc
+++ b/be/src/exprs/scalar-expr.cc
@@ -56,7 +56,6 @@
 #include "common/names.h"
 
 using namespace impala_udf;
-using namespace llvm;
 
 namespace impala {
 
@@ -331,7 +330,8 @@ int ScalarExpr::GetSlotIds(vector<SlotId>* slot_ids) const {
   return n;
 }
 
-Function* ScalarExpr::GetStaticGetValWrapper(ColumnType type, LlvmCodeGen* codegen) {
+llvm::Function* ScalarExpr::GetStaticGetValWrapper(
+    ColumnType type, LlvmCodeGen* codegen) {
   switch (type.type) {
     case TYPE_BOOLEAN:
       return codegen->GetFunction(IRFunction::SCALAR_EXPR_GET_BOOLEAN_VAL, false);
@@ -361,37 +361,38 @@ Function* ScalarExpr::GetStaticGetValWrapper(ColumnType type, LlvmCodeGen* codeg
   }
 }
 
-Function* ScalarExpr::CreateIrFunctionPrototype(const string& name,
-    LlvmCodeGen* codegen, Value* (*args)[2]) {
-  Type* return_type = CodegenAnyVal::GetLoweredType(codegen, type());
+llvm::Function* ScalarExpr::CreateIrFunctionPrototype(
+    const string& name, LlvmCodeGen* codegen, llvm::Value* (*args)[2]) {
+  llvm::Type* return_type = CodegenAnyVal::GetLoweredType(codegen, type());
   LlvmCodeGen::FnPrototype prototype(codegen, name, return_type);
   prototype.AddArgument(
       LlvmCodeGen::NamedVariable(
           "eval", codegen->GetPtrType(ScalarExprEvaluator::LLVM_CLASS_NAME)));
   prototype.AddArgument(
       LlvmCodeGen::NamedVariable("row", codegen->GetPtrType(TupleRow::LLVM_CLASS_NAME)));
-  Function* function = prototype.GeneratePrototype(NULL, args[0]);
+  llvm::Function* function = prototype.GeneratePrototype(NULL, args[0]);
   DCHECK(function != NULL);
   return function;
 }
 
-Status ScalarExpr::GetCodegendComputeFnWrapper(LlvmCodeGen* codegen, Function** fn) {
+Status ScalarExpr::GetCodegendComputeFnWrapper(
+    LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != nullptr) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
-  Function* static_getval_fn = GetStaticGetValWrapper(type(), codegen);
+  llvm::Function* static_getval_fn = GetStaticGetValWrapper(type(), codegen);
 
   // Call it passing this as the additional first argument.
-  Value* args[2];
+  llvm::Value* args[2];
   ir_compute_fn_ = CreateIrFunctionPrototype("CodegenComputeFnWrapper", codegen, &args);
-  BasicBlock* entry_block =
-      BasicBlock::Create(codegen->context(), "entry", ir_compute_fn_);
+  llvm::BasicBlock* entry_block =
+      llvm::BasicBlock::Create(codegen->context(), "entry", ir_compute_fn_);
   LlvmBuilder builder(entry_block);
-  Value* this_ptr =
+  llvm::Value* this_ptr =
       codegen->CastPtrToLlvmPtr(codegen->GetPtrType(ScalarExpr::LLVM_CLASS_NAME), this);
-  Value* compute_fn_args[] = {this_ptr, args[0], args[1]};
-  Value* ret = CodegenAnyVal::CreateCall(
+  llvm::Value* compute_fn_args[] = {this_ptr, args[0], args[1]};
+  llvm::Value* ret = CodegenAnyVal::CreateCall(
       codegen, &builder, static_getval_fn, compute_fn_args, "ret");
   builder.CreateRet(ret);
   *fn = codegen->FinalizeFunction(ir_compute_fn_);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/scalar-fn-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-fn-call.cc b/be/src/exprs/scalar-fn-call.cc
index 20dbe07..9e25175 100644
--- a/be/src/exprs/scalar-fn-call.cc
+++ b/be/src/exprs/scalar-fn-call.cc
@@ -43,13 +43,6 @@
 using namespace impala;
 using namespace impala_udf;
 using namespace strings;
-using llvm::ArrayType;
-using llvm::BasicBlock;
-using llvm::Function;
-using llvm::GlobalVariable;
-using llvm::PointerType;
-using llvm::Type;
-using llvm::Value;
 using std::move;
 using std::pair;
 
@@ -290,7 +283,7 @@ void ScalarFnCall::CloseEvaluator(FunctionContext::FunctionStateScope scope,
 //        i32 4,
 //        i64* inttoptr (i64 89111072 to i64*))
 //   ret { i8, double } %result
-Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
+Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
@@ -307,7 +300,7 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
 
   vector<ColumnType> arg_types;
   for (const Expr* child : children_) arg_types.push_back(child->type());
-  Function* udf;
+  llvm::Function* udf;
   RETURN_IF_ERROR(codegen->LoadFunction(fn_, fn_.scalar_fn.symbol, &type_, arg_types,
       NumFixedArgs(), vararg_start_idx_ != -1, &udf, &cache_entry_));
   // Inline constants into the function if it has an IR body.
@@ -331,32 +324,33 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
   stringstream fn_name;
   fn_name << udf->getName().str() << "Wrapper";
 
-  Value* args[2];
+  llvm::Value* args[2];
   *fn = CreateIrFunctionPrototype(fn_name.str(), codegen, &args);
-  Value* eval = args[0];
-  Value* row = args[1];
-  BasicBlock* block = BasicBlock::Create(codegen->context(), "entry", *fn);
+  llvm::Value* eval = args[0];
+  llvm::Value* row = args[1];
+  llvm::BasicBlock* block = llvm::BasicBlock::Create(codegen->context(), "entry", *fn);
   LlvmBuilder builder(block);
 
   // Populate UDF arguments
-  vector<Value*> udf_args;
+  vector<llvm::Value*> udf_args;
 
   // First argument is always FunctionContext*.
   // Index into our registered offset in the ScalarFnEvaluator.
-  Value* eval_gep = builder.CreateStructGEP(NULL, eval, 1, "eval_gep");
-  Value* fn_ctxs_base = builder.CreateLoad(eval_gep, "fn_ctxs_base");
+  llvm::Value* eval_gep = builder.CreateStructGEP(NULL, eval, 1, "eval_gep");
+  llvm::Value* fn_ctxs_base = builder.CreateLoad(eval_gep, "fn_ctxs_base");
   // Use GEP to add our index to the base pointer
-  Value* fn_ctx_ptr = builder.CreateConstGEP1_32(fn_ctxs_base, fn_ctx_idx_, "fn_ctx_ptr");
-  Value* fn_ctx = builder.CreateLoad(fn_ctx_ptr, "fn_ctx");
+  llvm::Value* fn_ctx_ptr =
+      builder.CreateConstGEP1_32(fn_ctxs_base, fn_ctx_idx_, "fn_ctx_ptr");
+  llvm::Value* fn_ctx = builder.CreateLoad(fn_ctx_ptr, "fn_ctx");
   udf_args.push_back(fn_ctx);
 
   // Allocate a varargs array. The array's entry type is the appropriate AnyVal subclass.
   // E.g. if the vararg type is STRING, and the function is called with 10 arguments, we
   // allocate a StringVal[10] array. We allocate the buffer with Alloca so that LLVM can
   // optimise out the buffer once the function call is inlined.
-  Value* varargs_buffer = NULL;
+  llvm::Value* varargs_buffer = NULL;
   if (vararg_start_idx_ != -1) {
-    Type* unlowered_varargs_type =
+    llvm::Type* unlowered_varargs_type =
         CodegenAnyVal::GetUnloweredType(codegen, VarArgsType());
     varargs_buffer = codegen->CreateEntryBlockAlloca(builder, unlowered_varargs_type,
         NumVarArgs(), FunctionContextImpl::VARARGS_BUFFER_ALIGNMENT, "varargs_buffer");
@@ -364,8 +358,8 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
 
   // Call children to populate remaining arguments
   for (int i = 0; i < GetNumChildren(); ++i) {
-    Function* child_fn = NULL;
-    vector<Value*> child_fn_args;
+    llvm::Function* child_fn = NULL;
+    vector<llvm::Value*> child_fn_args;
     // Set 'child_fn' to the codegen'd function, sets child_fn == NULL if codegen fails
     Status status = children_[i]->GetCodegendComputeFn(codegen, &child_fn);
     if (UNLIKELY(!status.ok())) {
@@ -373,7 +367,7 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
       // Set 'child_fn' to the interpreted function
       child_fn = GetStaticGetValWrapper(children_[i]->type(), codegen);
       // First argument to interpreted function is children_[i]
-      Type* expr_ptr_type = codegen->GetPtrType(ScalarExpr::LLVM_CLASS_NAME);
+      llvm::Type* expr_ptr_type = codegen->GetPtrType(ScalarExpr::LLVM_CLASS_NAME);
       child_fn_args.push_back(codegen->CastPtrToLlvmPtr(expr_ptr_type, children_[i]));
     }
     child_fn_args.push_back(eval);
@@ -381,8 +375,8 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
 
     // Call 'child_fn', adding the result to either 'udf_args' or 'varargs_buffer'
     DCHECK(child_fn != NULL);
-    Type* arg_type = CodegenAnyVal::GetUnloweredType(codegen, children_[i]->type());
-    Value* arg_val_ptr;
+    llvm::Type* arg_type = CodegenAnyVal::GetUnloweredType(codegen, children_[i]->type());
+    llvm::Value* arg_val_ptr;
     if (i < NumFixedArgs()) {
       // Allocate space to store 'child_fn's result so we can pass the pointer to the UDF.
       arg_val_ptr = codegen->CreateEntryBlockAlloca(builder, arg_type, "arg_val_ptr");
@@ -394,7 +388,7 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
     }
     DCHECK_EQ(arg_val_ptr->getType(), arg_type->getPointerTo());
     // The result of the call must be stored in a lowered AnyVal
-    Value* lowered_arg_val_ptr = builder.CreateBitCast(arg_val_ptr,
+    llvm::Value* lowered_arg_val_ptr = builder.CreateBitCast(arg_val_ptr,
         CodegenAnyVal::GetLoweredPtrType(codegen, children_[i]->type()),
         "lowered_arg_val_ptr");
     CodegenAnyVal::CreateCall(
@@ -408,12 +402,13 @@ Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
     // Add the number of varargs
     udf_args.push_back(codegen->GetIntConstant(TYPE_INT, NumVarArgs()));
     // Add all the accumulated vararg inputs as one input argument.
-    PointerType* vararg_type = CodegenAnyVal::GetUnloweredPtrType(codegen, VarArgsType());
+    llvm::PointerType* vararg_type =
+        CodegenAnyVal::GetUnloweredPtrType(codegen, VarArgsType());
     udf_args.push_back(builder.CreateBitCast(varargs_buffer, vararg_type, "varargs"));
   }
 
   // Call UDF
-  Value* result_val =
+  llvm::Value* result_val =
       CodegenAnyVal::CreateCall(codegen, &builder, udf, udf_args, "result");
   builder.CreateRet(result_val);
 
@@ -436,7 +431,7 @@ Status ScalarFnCall::GetFunction(LlvmCodeGen* codegen, const string& symbol, voi
   } else {
     DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::IR);
     DCHECK(codegen != NULL);
-    Function* ir_fn = codegen->GetFunction(symbol, false);
+    llvm::Function* ir_fn = codegen->GetFunction(symbol, false);
     if (ir_fn == NULL) {
       stringstream ss;
       ss << "Unable to locate function " << symbol << " from LLVM module "

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/exprs/slot-ref.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/slot-ref.cc b/be/src/exprs/slot-ref.cc
index 9f1a77d..d529013 100644
--- a/be/src/exprs/slot-ref.cc
+++ b/be/src/exprs/slot-ref.cc
@@ -35,7 +35,6 @@
 #include "common/names.h"
 
 using namespace impala_udf;
-using namespace llvm;
 
 namespace impala {
 
@@ -178,42 +177,44 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   int64_t unique_slot_id = slot_id_ | ((int64_t)tuple_idx_) << 32;
   DCHECK_EQ(unique_slot_id & TUPLE_NULLABLE_MASK, 0);
   if (tuple_is_nullable_) unique_slot_id |= TUPLE_NULLABLE_MASK;
-  Function* ir_compute_fn_ = codegen->GetRegisteredExprFn(unique_slot_id);
+  llvm::Function* ir_compute_fn_ = codegen->GetRegisteredExprFn(unique_slot_id);
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
-  LLVMContext& context = codegen->context();
-  Value* args[2];
+  llvm::LLVMContext& context = codegen->context();
+  llvm::Value* args[2];
   *fn = CreateIrFunctionPrototype("GetSlotRef", codegen, &args);
-  Value* row_ptr = args[1];
+  llvm::Value* row_ptr = args[1];
 
-  Value* tuple_offset = ConstantInt::get(codegen->int_type(), tuple_idx_);
-  Value* slot_offset = ConstantInt::get(codegen->int_type(), slot_offset_);
-  Value* zero = ConstantInt::get(codegen->GetType(TYPE_TINYINT), 0);
-  Value* one = ConstantInt::get(codegen->GetType(TYPE_TINYINT), 1);
+  llvm::Value* tuple_offset = llvm::ConstantInt::get(codegen->int_type(), tuple_idx_);
+  llvm::Value* slot_offset = llvm::ConstantInt::get(codegen->int_type(), slot_offset_);
+  llvm::Value* zero = llvm::ConstantInt::get(codegen->GetType(TYPE_TINYINT), 0);
+  llvm::Value* one = llvm::ConstantInt::get(codegen->GetType(TYPE_TINYINT), 1);
 
-  BasicBlock* entry_block = BasicBlock::Create(context, "entry", *fn);
+  llvm::BasicBlock* entry_block = llvm::BasicBlock::Create(context, "entry", *fn);
   bool slot_is_nullable = null_indicator_offset_.bit_mask != 0;
-  BasicBlock* check_slot_null_indicator_block = NULL;
+  llvm::BasicBlock* check_slot_null_indicator_block = NULL;
   if (slot_is_nullable) {
-    check_slot_null_indicator_block = BasicBlock::Create(context, "check_slot_null", *fn);
+    check_slot_null_indicator_block =
+        llvm::BasicBlock::Create(context, "check_slot_null", *fn);
   }
-  BasicBlock* get_slot_block = BasicBlock::Create(context, "get_slot", *fn);
-  BasicBlock* ret_block = BasicBlock::Create(context, "ret", *fn);
+  llvm::BasicBlock* get_slot_block = llvm::BasicBlock::Create(context, "get_slot", *fn);
+  llvm::BasicBlock* ret_block = llvm::BasicBlock::Create(context, "ret", *fn);
 
   LlvmBuilder builder(entry_block);
   // Get the tuple offset addr from the row
-  Value* cast_row_ptr = builder.CreateBitCast(
-      row_ptr, PointerType::get(codegen->ptr_type(), 0), "cast_row_ptr");
-  Value* tuple_addr = builder.CreateInBoundsGEP(cast_row_ptr, tuple_offset, "tuple_addr");
+  llvm::Value* cast_row_ptr = builder.CreateBitCast(
+      row_ptr, llvm::PointerType::get(codegen->ptr_type(), 0), "cast_row_ptr");
+  llvm::Value* tuple_addr =
+      builder.CreateInBoundsGEP(cast_row_ptr, tuple_offset, "tuple_addr");
   // Load the tuple*
-  Value* tuple_ptr = builder.CreateLoad(tuple_addr, "tuple_ptr");
+  llvm::Value* tuple_ptr = builder.CreateLoad(tuple_addr, "tuple_ptr");
 
   // Check if tuple* is null only if the tuple is nullable
   if (tuple_is_nullable_) {
-    Value* tuple_is_null = builder.CreateIsNull(tuple_ptr, "tuple_is_null");
+    llvm::Value* tuple_is_null = builder.CreateIsNull(tuple_ptr, "tuple_is_null");
     // Check slot is null only if the null indicator bit is set
     if (slot_is_nullable) {
       builder.CreateCondBr(tuple_is_null, ret_block, check_slot_null_indicator_block);
@@ -231,39 +232,42 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   // Branch for tuple* != NULL.  Need to check if null-indicator is set
   if (slot_is_nullable) {
     builder.SetInsertPoint(check_slot_null_indicator_block);
-    Value* is_slot_null = SlotDescriptor::CodegenIsNull(
+    llvm::Value* is_slot_null = SlotDescriptor::CodegenIsNull(
         codegen, &builder, null_indicator_offset_, tuple_ptr);
     builder.CreateCondBr(is_slot_null, ret_block, get_slot_block);
   }
 
   // Branch for slot != NULL
   builder.SetInsertPoint(get_slot_block);
-  Value* slot_ptr = builder.CreateInBoundsGEP(tuple_ptr, slot_offset, "slot_addr");
-  Value* val_ptr = builder.CreateBitCast(slot_ptr, codegen->GetPtrType(type_), "val_ptr");
+  llvm::Value* slot_ptr = builder.CreateInBoundsGEP(tuple_ptr, slot_offset, "slot_addr");
+  llvm::Value* val_ptr =
+      builder.CreateBitCast(slot_ptr, codegen->GetPtrType(type_), "val_ptr");
   // Depending on the type, load the values we need
-  Value* val = NULL;
-  Value* ptr = NULL;
-  Value* len = NULL;
-  Value* time_of_day = NULL;
-  Value* date = NULL;
+  llvm::Value* val = NULL;
+  llvm::Value* ptr = NULL;
+  llvm::Value* len = NULL;
+  llvm::Value* time_of_day = NULL;
+  llvm::Value* date = NULL;
   if (type_.IsStringType()) {
-    Value* ptr_ptr = builder.CreateStructGEP(NULL, val_ptr, 0, "ptr_ptr");
+    llvm::Value* ptr_ptr = builder.CreateStructGEP(NULL, val_ptr, 0, "ptr_ptr");
     ptr = builder.CreateLoad(ptr_ptr, "ptr");
-    Value* len_ptr = builder.CreateStructGEP(NULL, val_ptr, 1, "len_ptr");
+    llvm::Value* len_ptr = builder.CreateStructGEP(NULL, val_ptr, 1, "len_ptr");
     len = builder.CreateLoad(len_ptr, "len");
   } else if (type_.type == TYPE_FIXED_UDA_INTERMEDIATE) {
     // ptr and len are the slot and its fixed length.
     ptr = builder.CreateBitCast(val_ptr, codegen->ptr_type());
     len = codegen->GetIntConstant(TYPE_INT, type_.len);
   } else if (type_.type == TYPE_TIMESTAMP) {
-    Value* time_of_day_ptr = builder.CreateStructGEP(NULL, val_ptr, 0, "time_of_day_ptr");
+    llvm::Value* time_of_day_ptr =
+        builder.CreateStructGEP(NULL, val_ptr, 0, "time_of_day_ptr");
     // Cast boost::posix_time::time_duration to i64
-    Value* time_of_day_cast =
+    llvm::Value* time_of_day_cast =
         builder.CreateBitCast(time_of_day_ptr, codegen->GetPtrType(TYPE_BIGINT));
     time_of_day = builder.CreateLoad(time_of_day_cast, "time_of_day");
-    Value* date_ptr = builder.CreateStructGEP(NULL, val_ptr, 1, "date_ptr");
+    llvm::Value* date_ptr = builder.CreateStructGEP(NULL, val_ptr, 1, "date_ptr");
     // Cast boost::gregorian::date to i32
-    Value* date_cast = builder.CreateBitCast(date_ptr, codegen->GetPtrType(TYPE_INT));
+    llvm::Value* date_cast =
+        builder.CreateBitCast(date_ptr, codegen->GetPtrType(TYPE_INT));
     date = builder.CreateLoad(date_cast, "date");
   } else {
     // val_ptr is a native type
@@ -273,7 +277,8 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
 
   // Return block
   builder.SetInsertPoint(ret_block);
-  PHINode* is_null_phi = builder.CreatePHI(codegen->tinyint_type(), 2, "is_null_phi");
+  llvm::PHINode* is_null_phi =
+      builder.CreatePHI(codegen->tinyint_type(), 2, "is_null_phi");
   if (tuple_is_nullable_) is_null_phi->addIncoming(one, entry_block);
   if (check_slot_null_indicator_block != NULL) {
     is_null_phi->addIncoming(one, check_slot_null_indicator_block);
@@ -287,8 +292,8 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   if (type_.IsVarLenStringType() || type_.type == TYPE_FIXED_UDA_INTERMEDIATE) {
     DCHECK(ptr != NULL);
     DCHECK(len != NULL);
-    PHINode* ptr_phi = builder.CreatePHI(ptr->getType(), 2, "ptr_phi");
-    Value* null = Constant::getNullValue(ptr->getType());
+    llvm::PHINode* ptr_phi = builder.CreatePHI(ptr->getType(), 2, "ptr_phi");
+    llvm::Value* null = llvm::Constant::getNullValue(ptr->getType());
     if (tuple_is_nullable_) {
       ptr_phi->addIncoming(null, entry_block);
     }
@@ -297,8 +302,8 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
     }
     ptr_phi->addIncoming(ptr, get_slot_block);
 
-    PHINode* len_phi = builder.CreatePHI(len->getType(), 2, "len_phi");
-    null = ConstantInt::get(len->getType(), 0);
+    llvm::PHINode* len_phi = builder.CreatePHI(len->getType(), 2, "len_phi");
+    null = llvm::ConstantInt::get(len->getType(), 0);
     if (tuple_is_nullable_) {
       len_phi->addIncoming(null, entry_block);
     }
@@ -316,9 +321,9 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
   } else if (type_.type == TYPE_TIMESTAMP) {
     DCHECK(time_of_day != NULL);
     DCHECK(date != NULL);
-    PHINode* time_of_day_phi =
+    llvm::PHINode* time_of_day_phi =
         builder.CreatePHI(time_of_day->getType(), 2, "time_of_day_phi");
-    Value* null = ConstantInt::get(time_of_day->getType(), 0);
+    llvm::Value* null = llvm::ConstantInt::get(time_of_day->getType(), 0);
     if (tuple_is_nullable_) {
       time_of_day_phi->addIncoming(null, entry_block);
     }
@@ -327,8 +332,8 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
     }
     time_of_day_phi->addIncoming(time_of_day, get_slot_block);
 
-    PHINode* date_phi = builder.CreatePHI(date->getType(), 2, "date_phi");
-    null = ConstantInt::get(date->getType(), 0);
+    llvm::PHINode* date_phi = builder.CreatePHI(date->getType(), 2, "date_phi");
+    null = llvm::ConstantInt::get(date->getType(), 0);
     if (tuple_is_nullable_) {
       date_phi->addIncoming(null, entry_block);
     }
@@ -345,8 +350,8 @@ Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn)
     builder.CreateRet(result.GetLoweredValue());
   } else {
     DCHECK(val != NULL);
-    PHINode* val_phi = builder.CreatePHI(val->getType(), 2, "val_phi");
-    Value* null = Constant::getNullValue(val->getType());
+    llvm::PHINode* val_phi = builder.CreatePHI(val->getType(), 2, "val_phi");
+    llvm::Value* null = llvm::Constant::getNullValue(val->getType());
     if (tuple_is_nullable_) {
       val_phi->addIncoming(null, entry_block);
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/runtime/descriptors.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/descriptors.cc b/be/src/runtime/descriptors.cc
index 7f86145..6a4993b 100644
--- a/be/src/runtime/descriptors.cc
+++ b/be/src/runtime/descriptors.cc
@@ -37,11 +37,6 @@
 #include "common/names.h"
 
 using boost::algorithm::join;
-using llvm::Constant;
-using llvm::ConstantAggregateZero;
-using llvm::ConstantInt;
-using llvm::ConstantStruct;
-using llvm::StructType;
 using namespace strings;
 
 // In 'thrift_partition', the location is stored in a compressed format that references
@@ -79,15 +74,16 @@ string NullIndicatorOffset::DebugString() const {
   return out.str();
 }
 
-Constant* NullIndicatorOffset::ToIR(LlvmCodeGen* codegen) const {
-  StructType* null_indicator_offset_type =
-      static_cast<StructType*>(codegen->GetType(LLVM_CLASS_NAME));
+llvm::Constant* NullIndicatorOffset::ToIR(LlvmCodeGen* codegen) const {
+  llvm::StructType* null_indicator_offset_type =
+      static_cast<llvm::StructType*>(codegen->GetType(LLVM_CLASS_NAME));
   // Populate padding at end of struct with zeroes.
-  ConstantAggregateZero* zeroes = ConstantAggregateZero::get(null_indicator_offset_type);
-  return ConstantStruct::get(null_indicator_offset_type,
-      {ConstantInt::get(codegen->int_type(), byte_offset),
-      ConstantInt::get(codegen->tinyint_type(), bit_mask),
-      zeroes->getStructElement(2)});
+  llvm::ConstantAggregateZero* zeroes =
+      llvm::ConstantAggregateZero::get(null_indicator_offset_type);
+  return llvm::ConstantStruct::get(null_indicator_offset_type,
+      {llvm::ConstantInt::get(codegen->int_type(), byte_offset),
+          llvm::ConstantInt::get(codegen->tinyint_type(), bit_mask),
+          zeroes->getStructElement(2)});
 }
 
 ostream& operator<<(ostream& os, const NullIndicatorOffset& null_indicator) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index ac08c04..3415b0d 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -53,7 +53,6 @@
 
 #include "common/names.h"
 
-using namespace llvm;
 using strings::Substitute;
 
 DECLARE_int32(max_errors);
@@ -141,7 +140,7 @@ Status RuntimeState::CreateCodegen() {
 
 Status RuntimeState::CodegenScalarFns() {
   for (ScalarFnCall* scalar_fn : scalar_fns_to_codegen_) {
-    Function* fn;
+    llvm::Function* fn;
     RETURN_IF_ERROR(scalar_fn->GetCodegendComputeFn(codegen_.get(), &fn));
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/runtime/types.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/types.cc b/be/src/runtime/types.cc
index 1b21fe8..d2b215f 100644
--- a/be/src/runtime/types.cc
+++ b/be/src/runtime/types.cc
@@ -26,7 +26,6 @@
 #include "common/names.h"
 
 using namespace apache::hive::service::cli::thrift;
-using namespace llvm;
 
 namespace impala {
 
@@ -344,28 +343,32 @@ ostream& operator<<(ostream& os, const ColumnType& type) {
   return os;
 }
 
-ConstantStruct* ColumnType::ToIR(LlvmCodeGen* codegen) const {
+llvm::ConstantStruct* ColumnType::ToIR(LlvmCodeGen* codegen) const {
   // ColumnType = { i32, i32, i32, i32, <vector>, <vector> }
-  StructType* column_type_type = cast<StructType>(codegen->GetType(LLVM_CLASS_NAME));
+  llvm::StructType* column_type_type =
+      llvm::cast<llvm::StructType>(codegen->GetType(LLVM_CLASS_NAME));
 
   DCHECK_EQ(sizeof(type), sizeof(int32_t));
-  Constant* type_field = ConstantInt::get(codegen->int_type(), type);
+  llvm::Constant* type_field = llvm::ConstantInt::get(codegen->int_type(), type);
   DCHECK_EQ(sizeof(len), sizeof(int32_t));
-  Constant* len_field = ConstantInt::get(codegen->int_type(), len);
+  llvm::Constant* len_field = llvm::ConstantInt::get(codegen->int_type(), len);
   DCHECK_EQ(sizeof(precision), sizeof(int32_t));
-  Constant* precision_field = ConstantInt::get(codegen->int_type(), precision);
+  llvm::Constant* precision_field =
+      llvm::ConstantInt::get(codegen->int_type(), precision);
   DCHECK_EQ(sizeof(scale), sizeof(int32_t));
-  Constant* scale_field = ConstantInt::get(codegen->int_type(), scale);
+  llvm::Constant* scale_field = llvm::ConstantInt::get(codegen->int_type(), scale);
 
   // Create empty 'children' and 'field_names' vectors
   DCHECK(children.empty()) << "Nested types NYI";
   DCHECK(field_names.empty()) << "Nested types NYI";
-  Constant* children_field = Constant::getNullValue(column_type_type->getElementType(4));
-  Constant* field_names_field =
-      Constant::getNullValue(column_type_type->getElementType(5));
+  llvm::Constant* children_field =
+      llvm::Constant::getNullValue(column_type_type->getElementType(4));
+  llvm::Constant* field_names_field =
+      llvm::Constant::getNullValue(column_type_type->getElementType(5));
 
-  return cast<ConstantStruct>(ConstantStruct::get(column_type_type, type_field, len_field,
-      precision_field, scale_field, children_field, field_names_field, NULL));
+  return llvm::cast<llvm::ConstantStruct>(
+      llvm::ConstantStruct::get(column_type_type, type_field, len_field, precision_field,
+          scale_field, children_field, field_names_field, NULL));
 }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e98d2f1c/be/src/util/tuple-row-compare.cc
----------------------------------------------------------------------
diff --git a/be/src/util/tuple-row-compare.cc b/be/src/util/tuple-row-compare.cc
index 3d43576..ad92e84 100644
--- a/be/src/util/tuple-row-compare.cc
+++ b/be/src/util/tuple-row-compare.cc
@@ -27,7 +27,6 @@
 #include "util/runtime-profile-counters.h"
 
 using namespace impala;
-using namespace llvm;
 using namespace strings;
 
 Status TupleRowComparator::Open(ObjectPool* pool, RuntimeState* state,
@@ -73,7 +72,7 @@ int TupleRowComparator::CompareInterpreted(
 }
 
 Status TupleRowComparator::Codegen(RuntimeState* state) {
-  Function* fn;
+  llvm::Function* fn;
   LlvmCodeGen* codegen = state->codegen();
   DCHECK(codegen != NULL);
   RETURN_IF_ERROR(CodegenCompare(codegen, &fn));
@@ -203,11 +202,11 @@ Status TupleRowComparator::Codegen(RuntimeState* state) {
 // next_key2:                                        ; preds = %rhs_non_null12, %next_key
 //   ret i32 0
 // }
-Status TupleRowComparator::CodegenCompare(LlvmCodeGen* codegen, Function** fn) {
+Status TupleRowComparator::CodegenCompare(LlvmCodeGen* codegen, llvm::Function** fn) {
   SCOPED_TIMER(codegen->codegen_timer());
-  LLVMContext& context = codegen->context();
+  llvm::LLVMContext& context = codegen->context();
   const vector<ScalarExpr*>& ordering_exprs = ordering_exprs_;
-  Function* key_fns[ordering_exprs.size()];
+  llvm::Function* key_fns[ordering_exprs.size()];
   for (int i = 0; i < ordering_exprs.size(); ++i) {
     Status status = ordering_exprs[i]->GetCodegendComputeFn(codegen, &key_fns[i]);
     if (!status.ok()) {
@@ -221,9 +220,9 @@ Status TupleRowComparator::CodegenCompare(LlvmCodeGen* codegen, Function** fn) {
   // int Compare(ScalarExprEvaluator** ordering_expr_evals_lhs,
   //     ScalarExprEvaluator** ordering_expr_evals_rhs,
   //     TupleRow* lhs, TupleRow* rhs)
-  PointerType* expr_evals_type =
+  llvm::PointerType* expr_evals_type =
       codegen->GetPtrPtrType(ScalarExprEvaluator::LLVM_CLASS_NAME);
-  PointerType* tuple_row_type = codegen->GetPtrType(TupleRow::LLVM_CLASS_NAME);
+  llvm::PointerType* tuple_row_type = codegen->GetPtrType(TupleRow::LLVM_CLASS_NAME);
   LlvmCodeGen::FnPrototype prototype(codegen, "Compare", codegen->int_type());
   prototype.AddArgument("ordering_expr_evals_lhs", expr_evals_type);
   prototype.AddArgument("ordering_expr_evals_rhs", expr_evals_type);
@@ -231,69 +230,69 @@ Status TupleRowComparator::CodegenCompare(LlvmCodeGen* codegen, Function** fn) {
   prototype.AddArgument("rhs", tuple_row_type);
 
   LlvmBuilder builder(context);
-  Value* args[4];
+  llvm::Value* args[4];
   *fn = prototype.GeneratePrototype(&builder, args);
-  Value* lhs_evals_arg = args[0];
-  Value* rhs_evals_arg = args[1];
-  Value* lhs_arg = args[2];
-  Value* rhs_arg = args[3];
+  llvm::Value* lhs_evals_arg = args[0];
+  llvm::Value* rhs_evals_arg = args[1];
+  llvm::Value* lhs_arg = args[2];
+  llvm::Value* rhs_arg = args[3];
 
   // Unrolled loop over each key expr
   for (int i = 0; i < ordering_exprs.size(); ++i) {
     // The start of the next key expr after this one. Used to implement "continue" logic
     // in the unrolled loop.
-    BasicBlock* next_key_block = BasicBlock::Create(context, "next_key", *fn);
+    llvm::BasicBlock* next_key_block = llvm::BasicBlock::Create(context, "next_key", *fn);
 
     // Call key_fns[i](ordering_expr_evals_lhs[i], lhs_arg)
-    Value* lhs_eval = codegen->CodegenArrayAt(&builder, lhs_evals_arg, i);
-    Value* lhs_args[] = { lhs_eval, lhs_arg };
+    llvm::Value* lhs_eval = codegen->CodegenArrayAt(&builder, lhs_evals_arg, i);
+    llvm::Value* lhs_args[] = {lhs_eval, lhs_arg};
     CodegenAnyVal lhs_value = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
         ordering_exprs[i]->type(), key_fns[i], lhs_args, "lhs_value");
 
     // Call key_fns[i](ordering_expr_evals_rhs[i], rhs_arg)
-    Value* rhs_eval = codegen->CodegenArrayAt(&builder, rhs_evals_arg, i);
-    Value* rhs_args[] = { rhs_eval, rhs_arg };
+    llvm::Value* rhs_eval = codegen->CodegenArrayAt(&builder, rhs_evals_arg, i);
+    llvm::Value* rhs_args[] = {rhs_eval, rhs_arg};
     CodegenAnyVal rhs_value = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
         ordering_exprs[i]->type(), key_fns[i], rhs_args, "rhs_value");
 
     // Handle NULLs if necessary
-    Value* lhs_null = lhs_value.GetIsNull();
-    Value* rhs_null = rhs_value.GetIsNull();
+    llvm::Value* lhs_null = lhs_value.GetIsNull();
+    llvm::Value* rhs_null = rhs_value.GetIsNull();
     // if (lhs_value == NULL && rhs_value == NULL) continue;
-    Value* both_null = builder.CreateAnd(lhs_null, rhs_null, "both_null");
-    BasicBlock* non_null_block =
-        BasicBlock::Create(context, "non_null", *fn, next_key_block);
+    llvm::Value* both_null = builder.CreateAnd(lhs_null, rhs_null, "both_null");
+    llvm::BasicBlock* non_null_block =
+        llvm::BasicBlock::Create(context, "non_null", *fn, next_key_block);
     builder.CreateCondBr(both_null, next_key_block, non_null_block);
     // if (lhs_value == NULL && rhs_value != NULL) return nulls_first_[i];
     builder.SetInsertPoint(non_null_block);
-    BasicBlock* lhs_null_block =
-        BasicBlock::Create(context, "lhs_null", *fn, next_key_block);
-    BasicBlock* lhs_non_null_block =
-        BasicBlock::Create(context, "lhs_non_null", *fn, next_key_block);
+    llvm::BasicBlock* lhs_null_block =
+        llvm::BasicBlock::Create(context, "lhs_null", *fn, next_key_block);
+    llvm::BasicBlock* lhs_non_null_block =
+        llvm::BasicBlock::Create(context, "lhs_non_null", *fn, next_key_block);
     builder.CreateCondBr(lhs_null, lhs_null_block, lhs_non_null_block);
     builder.SetInsertPoint(lhs_null_block);
     builder.CreateRet(builder.getInt32(nulls_first_[i]));
     // if (lhs_value != NULL && rhs_value == NULL) return -nulls_first_[i];
     builder.SetInsertPoint(lhs_non_null_block);
-    BasicBlock* rhs_null_block =
-        BasicBlock::Create(context, "rhs_null", *fn, next_key_block);
-    BasicBlock* rhs_non_null_block =
-        BasicBlock::Create(context, "rhs_non_null", *fn, next_key_block);
+    llvm::BasicBlock* rhs_null_block =
+        llvm::BasicBlock::Create(context, "rhs_null", *fn, next_key_block);
+    llvm::BasicBlock* rhs_non_null_block =
+        llvm::BasicBlock::Create(context, "rhs_non_null", *fn, next_key_block);
     builder.CreateCondBr(rhs_null, rhs_null_block, rhs_non_null_block);
     builder.SetInsertPoint(rhs_null_block);
     builder.CreateRet(builder.getInt32(-nulls_first_[i]));
 
     // int result = RawValue::Compare(lhs_value, rhs_value, <type>)
     builder.SetInsertPoint(rhs_non_null_block);
-    Value* result = lhs_value.Compare(&rhs_value, "result");
+    llvm::Value* result = lhs_value.Compare(&rhs_value, "result");
 
     // if (!is_asc_[i]) result = -result;
     if (!is_asc_[i]) result = builder.CreateSub(builder.getInt32(0), result, "result");
     // if (result != 0) return result;
     // Otherwise, try the next Expr
-    Value* result_nonzero = builder.CreateICmpNE(result, builder.getInt32(0));
-    BasicBlock* result_nonzero_block =
-        BasicBlock::Create(context, "result_nonzero", *fn, next_key_block);
+    llvm::Value* result_nonzero = builder.CreateICmpNE(result, builder.getInt32(0));
+    llvm::BasicBlock* result_nonzero_block =
+        llvm::BasicBlock::Create(context, "result_nonzero", *fn, next_key_block);
     builder.CreateCondBr(result_nonzero, result_nonzero_block, next_key_block);
     builder.SetInsertPoint(result_nonzero_block);
     builder.CreateRet(result);