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/09/16 21:41:02 UTC

[3/5] incubator-impala git commit: IMPALA-3877: support unpatched LLVM

IMPALA-3877: support unpatched LLVM

The p1 patch we use for LLVM avoided merging of structurally
identical Struct types in unpredictable ways when linking in
IR UDF modules. This avoided hitting type assertions when
generating calls to IR UDfs.

This implements an alternative solution, which is to bitcast
the arguments when calling IR UDFs. This means we do not need to carry
the patch when we upgrade LLVM.

Testing:
Ran core tests with unpatched LLVM 3.8, including the IR UDF test that
originally required the patch to pass.

Change-Id: I3dfbe44ed8a1464b9b0991fd54e72b194ad6155d
Reviewed-on: http://gerrit.cloudera.org:8080/7973
Reviewed-by: Tim Armstrong <ta...@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/491822f0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/491822f0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/491822f0

Branch: refs/heads/master
Commit: 491822f0e3c8e5b4890533924d72375caad4bc74
Parents: 02302b7
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Aug 25 11:54:14 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sat Sep 16 00:46:05 2017 +0000

----------------------------------------------------------------------
 be/src/codegen/CMakeLists.txt     |  1 +
 be/src/codegen/codegen-anyval.cc  | 14 +++++--
 be/src/codegen/codegen-util.cc    | 76 ++++++++++++++++++++++++++++++++++
 be/src/codegen/codegen-util.h     | 65 +++++++++++++++++++++++++++++
 be/src/codegen/llvm-codegen.h     |  4 +-
 be/src/exprs/expr-codegen-test.cc |  3 +-
 6 files changed, 158 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/be/src/codegen/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/codegen/CMakeLists.txt b/be/src/codegen/CMakeLists.txt
index e640009..56228a2 100644
--- a/be/src/codegen/CMakeLists.txt
+++ b/be/src/codegen/CMakeLists.txt
@@ -30,6 +30,7 @@ add_library(CodeGen
   codegen-anyval.cc
   codegen-callgraph.cc
   codegen-symbol-emitter.cc
+  codegen-util.cc
   llvm-codegen.cc
   instruction-counter.cc
   ${IR_SSE_C_FILE}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/be/src/codegen/codegen-anyval.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.cc b/be/src/codegen/codegen-anyval.cc
index bdd49c9..b116cfe 100644
--- a/be/src/codegen/codegen-anyval.cc
+++ b/be/src/codegen/codegen-anyval.cc
@@ -17,6 +17,8 @@
 
 #include "codegen/codegen-anyval.h"
 
+#include "codegen/codegen-util.h"
+
 #include "common/names.h"
 
 using namespace impala;
@@ -138,15 +140,21 @@ Value* CodegenAnyVal::CreateCall(LlvmCodeGen* cg, LlvmBuilder* builder, Function
                      cg->CreateEntryBlockAlloca(*builder, ret_type, name) : result_ptr;
     vector<Value*> new_args = args.vec();
     new_args.insert(new_args.begin(), ret_ptr);
-    builder->CreateCall(fn, new_args);
+    // Bitcasting the args is often necessary when calling an IR UDF because the types
+    // in the IR module may have been renamed while linking. Bitcasting them avoids a
+    // type assertion.
+    CodeGenUtil::CreateCallWithBitCasts(builder, fn, new_args);
 
     // If 'result_ptr' was specified, we're done. Otherwise load and return the result.
     if (result_ptr != NULL) return NULL;
     return builder->CreateLoad(ret_ptr, name);
   } else {
     // Function returns *Val normally (note that it could still be returning a DecimalVal,
-    // since we generate non-complaint functions)
-    Value* ret = builder->CreateCall(fn, args, name);
+    // since we generate non-compliant functions).
+    // Bitcasting the args is often necessary when calling an IR UDF because the types
+    // in the IR module may have been renamed while linking. Bitcasting them avoids a
+    // type assertion.
+    Value* ret = CodeGenUtil::CreateCallWithBitCasts(builder, fn, args, name);
     if (result_ptr == NULL) return ret;
     builder->CreateStore(ret, result_ptr);
     return NULL;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/be/src/codegen/codegen-util.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-util.cc b/be/src/codegen/codegen-util.cc
new file mode 100644
index 0000000..9890ac2
--- /dev/null
+++ b/be/src/codegen/codegen-util.cc
@@ -0,0 +1,76 @@
+// 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.
+
+#include "codegen/codegen-util.h"
+
+#include <cctype>
+
+#include "common/names.h"
+
+using namespace llvm;
+using std::isdigit;
+
+namespace impala {
+
+CallInst* CodeGenUtil::CreateCallWithBitCasts(LlvmBuilder* builder,
+    Function *callee, ArrayRef<Value*> args, const Twine& name) {
+  vector<Value*> bitcast_args;
+  bitcast_args.reserve(args.size());
+  Function::arg_iterator fn_arg = callee->arg_begin();
+  for (Value* arg: args) {
+    bitcast_args.push_back(
+        CheckedBitCast(builder, arg, fn_arg->getType(), "create_call_bitcast"));
+    ++fn_arg;
+  }
+  return builder->CreateCall(callee, bitcast_args, name);
+}
+
+Value* CodeGenUtil::CheckedBitCast(LlvmBuilder* builder, Value* value,
+    Type* dst_type, const Twine& name) {
+  DCHECK(TypesAreStructurallyIdentical(value->getType(), dst_type))
+      << Print(value->getType()) << " " << Print(dst_type);
+  return builder->CreateBitCast(value, dst_type, name);
+}
+
+bool CodeGenUtil::TypesAreStructurallyIdentical(Type* t1, Type* t2) {
+  // All primitive types are deduplicated by LLVM, so we can just compare the pointers.
+  if (t1 == t2) return true;
+  // Derived types are structurally identical if they are the same kind of compound type
+  // and the elements are structurally identical. Check to see which of the Type
+  // subclasses t1 belongs to.
+  if (t1->isPointerTy()) {
+    if (!t2->isPointerTy()) return false;
+  } else if (t1->isStructTy()) {
+    if (!t2->isStructTy()) return false;
+  } else if (t1->isArrayTy()) {
+    if (!t2->isArrayTy()) return false;
+  } else if (t1->isFunctionTy()) {
+    if (!t2->isFunctionTy()) return false;
+  } else {
+    DCHECK(t1->isVectorTy()) << Print(t1);
+    if (!t2->isVectorTy()) return false;
+  }
+  if (t1->getNumContainedTypes() != t2->getNumContainedTypes()) return false;
+  for (int i = 0; i < t1->getNumContainedTypes(); ++i) {
+    if (!TypesAreStructurallyIdentical(
+          t1->getContainedType(i), t2->getContainedType(i))) {
+      return false;
+    }
+  }
+  return true;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/be/src/codegen/codegen-util.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-util.h b/be/src/codegen/codegen-util.h
new file mode 100644
index 0000000..0a2199b
--- /dev/null
+++ b/be/src/codegen/codegen-util.h
@@ -0,0 +1,65 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_CODEGEN_CODEGEN_UTIL_H
+#define IMPALA_CODEGEN_CODEGEN_UTIL_H
+
+#include <vector>
+#include <llvm/IR/IRBuilder.h>
+
+#include "codegen/llvm-codegen.h"
+
+namespace impala {
+
+/// Miscellaneous codegen utility functions that don't depend on the rest of the Impala
+/// codegen infrastructure.
+class CodeGenUtil {
+ public:
+  /// Wrapper around IRBuilder::CreateCall() that automatically bitcasts arguments
+  /// using CheckedBitCast(). This should be used instead of IRBuilder::CreateCall()
+  /// when calling functions from a linked module because the LLVM linker may merge
+  /// different struct types with the same memory layout during linking. E.g. if the
+  /// IR UDF module has type TinyIntVal that has the same memory layout as BooleanVal:
+  /// {i8, i8}, then the linker may substitute references to TinyIntVal with BooleanVal
+  /// in the IR UDF. Calling a function which has a BooleanVal* argument with a TinyInt*
+  /// argument without bitcasting then would result in hitting an internal LLVM assertion.
+  static llvm::CallInst* CreateCallWithBitCasts(LlvmBuilder* builder,
+      llvm::Function* callee, llvm::ArrayRef<llvm::Value*> args,
+      const llvm::Twine& name="");
+
+  /// Same as IRBuilder::CreateBitCast() except that it checks that the types are either
+  /// exactly the same, or, if they are both struct types (or pointers to struct types),
+  /// that they are structurally identical. Either returns 'value' if no conversion is
+  /// necessary, or returns a bitcast instruction converting 'value' to 'dst_type'.
+  static llvm::Value* CheckedBitCast(LlvmBuilder* builder,
+      llvm::Value* value, llvm::Type* dst_type, const llvm::Twine& name);
+
+  /// Return true if the two types are structurally identical.
+  static bool TypesAreStructurallyIdentical(llvm::Type *t1, llvm::Type *t2);
+
+  /// Returns the string representation of a llvm::Value* or llvm::Type*
+  template <typename T> static std::string Print(T* value_or_type) {
+    std::string str;
+    llvm::raw_string_ostream stream(str);
+    value_or_type->print(stream);
+    return str;
+  }
+
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index dca344a..ea31372 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -310,7 +310,9 @@ class LlvmCodeGen {
       LibCacheEntry** cache_entry);
 
   /// Replaces all instructions in 'caller' that call 'target_name' with a call
-  /// instruction to 'new_fn'. Returns the number of call sites updated.
+  /// instruction to 'new_fn'. The argument types of 'new_fn' must exactly match
+  /// the argument types of the function to be replaced. Returns the number of
+  /// call sites updated.
   ///
   /// 'target_name' must be a substring of the mangled symbol of the function to be
   /// replaced. This usually means that the unmangled function name is sufficient.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/491822f0/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 9ea865a..4b13073 100644
--- a/be/src/exprs/expr-codegen-test.cc
+++ b/be/src/exprs/expr-codegen-test.cc
@@ -55,6 +55,7 @@ DecimalVal TestGetFnAttrs(
 #ifndef IR_COMPILE
 
 #include "testutil/gtest-util.h"
+#include "codegen/codegen-util.h"
 #include "codegen/llvm-codegen.h"
 #include "common/init.h"
 #include "exprs/anyval-util.h"
@@ -328,7 +329,7 @@ TEST_F(ExprCodegenTest, TestInlineConstFnAttrs) {
   EXPECT_EQ(replaced, 9);
   ResetVerification(codegen.get());
   verification_succeeded = VerifyFunction(codegen.get(), fn);
-  EXPECT_TRUE(verification_succeeded) << LlvmCodeGen::Print(fn);
+  EXPECT_TRUE(verification_succeeded) << CodeGenUtil::Print(fn);
 
   // Compile module
   fn = codegen->FinalizeFunction(fn);