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 2019/06/25 15:45:25 UTC

[impala] 13/20: IMPALA-6299: Use LlvmCodeGen's internal list of white-listed CPU attributes for handcrafting IRs

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch 2.x
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 44794423bada8a44a2b7b05879764c6bc13b26fd
Author: poojanilangekar <po...@cloudera.com>
AuthorDate: Thu Jul 12 15:09:17 2018 -0700

    IMPALA-6299: Use LlvmCodeGen's internal list of white-listed CPU attributes for handcrafting IRs
    
    Previously, the IRBuilder of the LlvmCodeGen class used CpuInfo's
    list of enabled features to determine the validity of certain
    instructions to emit intrinsics. It did not consider the whitelist
    which passed while initializing the LlvmCodeGen class. Now, the
    IRBuilder inspects its own CPU attributes before emitting
    instruction. This change also adds functionality to modify the CPU
    attributes of the LlvmCodeGen class for testing.
    
    Testing: Verified that the current tests which use and modify
    CpuInfo produce expected results.
    
    Change-Id: Ifece8949c143146d2a1b38d72d21c2d733bed90f
    Reviewed-on: http://gerrit.cloudera.org:8080/10979
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/bloom-filter-benchmark.cc |  2 ++
 be/src/codegen/llvm-codegen-test.cc         | 49 ++++++++++++++++++++++-------
 be/src/codegen/llvm-codegen.cc              | 39 ++++++++++++++++-------
 be/src/codegen/llvm-codegen.h               | 24 ++++++++++----
 be/src/exec/filter-context.cc               |  2 +-
 be/src/util/cpu-info.cc                     |  8 +++++
 6 files changed, 95 insertions(+), 29 deletions(-)

diff --git a/be/src/benchmarks/bloom-filter-benchmark.cc b/be/src/benchmarks/bloom-filter-benchmark.cc
index 4657a18..f216911 100644
--- a/be/src/benchmarks/bloom-filter-benchmark.cc
+++ b/be/src/benchmarks/bloom-filter-benchmark.cc
@@ -48,6 +48,8 @@ using namespace impala;
 // As in bloom-filter.h, ndv refers to the number of unique items inserted into a filter
 // and fpp is the probability of false positives.
 //
+// This benchmark must be executed only in RELEASE mode. Since it executes some codepath
+// which would not occur in Impala's execution, it crashes due to a DCHECK in DEBUG mode.
 //
 // Machine Info: Intel(R) Core(TM) i7-6700 CPU @ 3.40GHz
 //
diff --git a/be/src/codegen/llvm-codegen-test.cc b/be/src/codegen/llvm-codegen-test.cc
index 9f668e0..3748ced 100644
--- a/be/src/codegen/llvm-codegen-test.cc
+++ b/be/src/codegen/llvm-codegen-test.cc
@@ -111,6 +111,31 @@ class LlvmCodeGenTest : public testing:: Test {
     const auto& hf = codegen->handcrafted_functions_;
     return find(hf.begin(), hf.end(), function) != hf.end();
   }
+
+  // Helper function to enable and disable LlvmCodeGen's cpu_attrs_.
+  static void EnableCodeGenCPUFeature(int64_t flag, bool enable) {
+    DCHECK(LlvmCodeGen::llvm_initialized_);
+    auto enable_flag_it = LlvmCodeGen::cpu_flag_mappings_.find(flag);
+    auto disable_flag_it = LlvmCodeGen::cpu_flag_mappings_.find(~flag);
+    DCHECK(enable_flag_it != LlvmCodeGen::cpu_flag_mappings_.end());
+    DCHECK(disable_flag_it != LlvmCodeGen::cpu_flag_mappings_.end());
+    const std::string& enable_feature = enable_flag_it->second;
+    const std::string& disable_feature = disable_flag_it->second;
+    if (enable) {
+      auto attr_it = LlvmCodeGen::cpu_attrs_.find(disable_feature);
+      if (attr_it != LlvmCodeGen::cpu_attrs_.end()) {
+        LlvmCodeGen::cpu_attrs_.erase(attr_it);
+      }
+      LlvmCodeGen::cpu_attrs_.insert(enable_feature);
+    } else {
+      auto attr_it = LlvmCodeGen::cpu_attrs_.find(enable_feature);
+      // Disable feature if currently enabled.
+      if (attr_it != LlvmCodeGen::cpu_attrs_.end()) {
+        LlvmCodeGen::cpu_attrs_.erase(attr_it);
+        LlvmCodeGen::cpu_attrs_.insert(disable_feature);
+      }
+    }
+  }
 };
 
 // Simple test to just make and destroy llvmcodegen objects.  LLVM
@@ -460,9 +485,12 @@ TEST_F(LlvmCodeGenTest, HashTest) {
     uint32_t result = test_fn();
 
     // Validate that the hashes are identical
-    EXPECT_EQ(result, expected_hash) << CpuInfo::IsSupported(CpuInfo::SSE4_2);
+    EXPECT_EQ(result, expected_hash) << LlvmCodeGen::IsCPUFeatureEnabled(CpuInfo::SSE4_2);
 
-    if (i == 0 && CpuInfo::IsSupported(CpuInfo::SSE4_2)) {
+    if (i == 0 && LlvmCodeGen::IsCPUFeatureEnabled(CpuInfo::SSE4_2)) {
+      // Modify both CpuInfo and LlvmCodeGen::cpu_attrs_ to ensure that they have the
+      // same view of the underlying hardware while generating the hash.
+      EnableCodeGenCPUFeature(CpuInfo::SSE4_2, false);
       CpuInfo::EnableFeature(CpuInfo::SSE4_2, false);
       restore_sse_support = true;
       LlvmCodeGenTest::ClearHashFns(codegen.get());
@@ -473,6 +501,7 @@ TEST_F(LlvmCodeGenTest, HashTest) {
   }
 
   // Restore hardware feature for next test
+  EnableCodeGenCPUFeature(CpuInfo::SSE4_2, restore_sse_support);
   CpuInfo::EnableFeature(CpuInfo::SSE4_2, restore_sse_support);
 }
 
@@ -495,19 +524,17 @@ TEST_F(LlvmCodeGenTest, CpuAttrWhitelist) {
   // Non-existent attributes should be disabled regardless of initial states.
   // Whitelisted attributes like sse2 and lzcnt should retain their initial
   // state.
-  EXPECT_EQ(vector<string>(
-          {"-dummy1", "-dummy2", "-dummy3", "-dummy4", "+sse2", "-lzcnt"}),
+  EXPECT_EQ(std::unordered_set<string>(
+                {"-dummy1", "-dummy2", "-dummy3", "-dummy4", "+sse2", "-lzcnt"}),
       LlvmCodeGen::ApplyCpuAttrWhitelist(
-          {"+dummy1", "+dummy2", "-dummy3", "+dummy4", "+sse2", "-lzcnt"}));
+                {"+dummy1", "+dummy2", "-dummy3", "+dummy4", "+sse2", "-lzcnt"}));
 
   // IMPALA-6291: Test that all AVX512 attributes are disabled.
   vector<string> avx512_attrs;
-  EXPECT_EQ(vector<string>(
-        {"-avx512ifma", "-avx512dqavx512er", "-avx512f", "-avx512bw", "-avx512vl",
-         "-avx512cd", "-avx512vbmi", "-avx512pf"}),
-      LlvmCodeGen::ApplyCpuAttrWhitelist(
-        {"+avx512ifma", "+avx512dqavx512er", "+avx512f", "+avx512bw", "+avx512vl",
-         "+avx512cd", "+avx512vbmi", "+avx512pf"}));
+  EXPECT_EQ(std::unordered_set<string>({"-avx512ifma", "-avx512dqavx512er", "-avx512f",
+                "-avx512bw", "-avx512vl", "-avx512cd", "-avx512vbmi", "-avx512pf"}),
+      LlvmCodeGen::ApplyCpuAttrWhitelist({"+avx512ifma", "+avx512dqavx512er", "+avx512f",
+          "+avx512bw", "+avx512vl", "+avx512cd", "+avx512vbmi", "+avx512pf"}));
 }
 
 // Test that exercises the code path that deletes non-finalized methods before it
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 173dbb2..d6f89a3 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -119,10 +119,19 @@ namespace impala {
 
 bool LlvmCodeGen::llvm_initialized_ = false;
 string LlvmCodeGen::cpu_name_;
-vector<string> LlvmCodeGen::cpu_attrs_;
+std::unordered_set<string> LlvmCodeGen::cpu_attrs_;
 string LlvmCodeGen::target_features_attr_;
 CodegenCallGraph LlvmCodeGen::shared_call_graph_;
 
+const map<int64_t, std::string> LlvmCodeGen::cpu_flag_mappings_{
+    {CpuInfo::SSSE3, "+ssse3"}, {CpuInfo::SSE4_1, "+sse4.1"},
+    {CpuInfo::SSE4_2, "+sse4.2"}, {CpuInfo::POPCNT, "+popcnt"}, {CpuInfo::AVX, "+avx"},
+    {CpuInfo::AVX2, "+avx2"}, {CpuInfo::PCLMULQDQ, "+pclmul"},
+    {~(CpuInfo::SSSE3), "-ssse3"}, {~(CpuInfo::SSE4_1), "-sse4.1"},
+    {~(CpuInfo::SSE4_2), "-sse4.2"}, {~(CpuInfo::POPCNT), "-popcnt"},
+    {~(CpuInfo::AVX), "-avx"}, {~(CpuInfo::AVX2), "-avx2"},
+    {~(CpuInfo::PCLMULQDQ), "-pclmul"}};
+
 [[noreturn]] static void LlvmCodegenHandleError(
     void* user_data, const string& reason, bool gen_crash_diag) {
   LOG(FATAL) << "LLVM hit fatal error: " << reason.c_str();
@@ -238,7 +247,7 @@ Status LlvmCodeGen::CreateFromMemory(RuntimeState* state, ObjectPool* pool,
   // a machine without SSE4.2 support.
   llvm::StringRef module_ir;
   string module_name;
-  if (CpuInfo::IsSupported(CpuInfo::SSE4_2)) {
+  if (IsCPUFeatureEnabled(CpuInfo::SSE4_2)) {
     module_ir = llvm::StringRef(
         reinterpret_cast<const char*>(impala_sse_llvm_ir), impala_sse_llvm_ir_len);
     module_name = "Impala IR with SSE 4.2 support";
@@ -471,17 +480,24 @@ void LlvmCodeGen::EnableOptimizations(bool enable) {
   optimizations_enabled_ = enable;
 }
 
-void LlvmCodeGen::GetHostCPUAttrs(vector<string>* attrs) {
+void LlvmCodeGen::GetHostCPUAttrs(std::unordered_set<string>* attrs) {
   // LLVM's ExecutionEngine expects features to be enabled or disabled with a list
   // of strings like ["+feature1", "-feature2"].
   llvm::StringMap<bool> cpu_features;
   llvm::sys::getHostCPUFeatures(cpu_features);
   for (const llvm::StringMapEntry<bool>& entry : cpu_features) {
-    attrs->emplace_back(
-        Substitute("$0$1", entry.second ? "+" : "-", entry.first().data()));
+    attrs->emplace(Substitute("$0$1", entry.second ? "+" : "-", entry.first().data()));
   }
 }
 
+bool LlvmCodeGen::IsCPUFeatureEnabled(int64_t flag) {
+  DCHECK(llvm_initialized_);
+  auto enable_flag_it = cpu_flag_mappings_.find(flag);
+  DCHECK(enable_flag_it != cpu_flag_mappings_.end());
+  const std::string& enabled_feature = enable_flag_it->second;
+  return cpu_attrs_.find(enabled_feature) != cpu_attrs_.end();
+}
+
 string LlvmCodeGen::GetIR(bool full_module) const {
   string str;
   llvm::raw_string_ostream stream(str);
@@ -1526,7 +1542,7 @@ void LlvmCodeGen::ClearHashFns() {
 //   ret i32 %12
 // }
 llvm::Function* LlvmCodeGen::GetHashFunction(int num_bytes) {
-  if (CpuInfo::IsSupported(CpuInfo::SSE4_2)) {
+  if (IsCPUFeatureEnabled(CpuInfo::SSE4_2)) {
     if (num_bytes == -1) {
       // -1 indicates variable length, just return the generic loop based
       // hash fn.
@@ -1671,8 +1687,9 @@ llvm::Constant* LlvmCodeGen::ConstantsToGVArrayPtr(llvm::Type* element_type,
   return ConstantToGVPtr(array_type, array_const, name);
 }
 
-vector<string> LlvmCodeGen::ApplyCpuAttrWhitelist(const vector<string>& cpu_attrs) {
-  vector<string> result;
+std::unordered_set<string> LlvmCodeGen::ApplyCpuAttrWhitelist(
+    const std::unordered_set<string>& cpu_attrs) {
+  std::unordered_set<string> result;
   vector<string> attr_whitelist;
   boost::split(attr_whitelist, FLAGS_llvm_cpu_attr_whitelist, boost::is_any_of(","));
   for (const string& attr : cpu_attrs) {
@@ -1680,17 +1697,17 @@ vector<string> LlvmCodeGen::ApplyCpuAttrWhitelist(const vector<string>& cpu_attr
     DCHECK(attr[0] == '-' || attr[0] == '+') << attr;
     if (attr[0] == '-') {
       // Already disabled - copy it over unmodified.
-      result.push_back(attr);
+      result.insert(attr);
       continue;
     }
     const string attr_name = attr.substr(1);
     auto it = std::find(attr_whitelist.begin(), attr_whitelist.end(), attr_name);
     if (it != attr_whitelist.end()) {
       // In whitelist - copy it over unmodified.
-      result.push_back(attr);
+      result.insert(attr);
     } else {
       // Not in whitelist - disable it.
-      result.push_back("-" + attr_name);
+      result.insert("-" + attr_name);
     }
   }
   return result;
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 7e9da26..9ede053 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -238,7 +238,10 @@ class LlvmCodeGen {
   };
 
   /// Get host cpu attributes in format expected by EngineBuilder.
-  static void GetHostCPUAttrs(std::vector<std::string>* attrs);
+  static void GetHostCPUAttrs(std::unordered_set<std::string>* attrs);
+
+  /// Returns whether or not this cpu feature is supported.
+  static bool IsCPUFeatureEnabled(int64_t flag);
 
   /// Return a pointer type to 'type'
   llvm::PointerType* GetPtrType(llvm::Type* type);
@@ -593,6 +596,7 @@ class LlvmCodeGen {
   friend class ExprCodegenTest;
   friend class LlvmCodeGenTest;
   friend class LlvmCodeGenTest_CpuAttrWhitelist_Test;
+  friend class LlvmCodeGenTest_HashTest_Test;
   friend class SubExprElimination;
 
   /// Top level codegen object. 'module_id' is used for debugging when outputting the IR.
@@ -714,21 +718,29 @@ class LlvmCodeGen {
   /// always present in the output, except "+" is flipped to "-" for the disabled
   /// attributes. E.g. if 'cpu_attrs' is {"+x", "+y", "-z"} and the whitelist is
   /// {"x", "z"}, returns {"+x", "-y", "-z"}.
-  static std::vector<std::string> ApplyCpuAttrWhitelist(
-      const std::vector<std::string>& cpu_attrs);
+  static std::unordered_set<std::string> ApplyCpuAttrWhitelist(
+      const std::unordered_set<std::string>& cpu_attrs);
 
   /// Whether InitializeLlvm() has been called.
   static bool llvm_initialized_;
 
   /// Host CPU name and attributes, filled in by InitializeLlvm().
   static std::string cpu_name_;
-  static std::vector<std::string> cpu_attrs_;
+  /// The cpu_attrs_ should not be modified during the execution except for tests.
+  static std::unordered_set<std::string> cpu_attrs_;
 
   /// Value of "target-features" attribute to be set on all IR functions. Derived from
-  /// 'cpu_attrs_'. Using a consistent value for this attribute among hand-crafted IR
-  /// and cross-compiled functions allow them to be inlined into each other.
+  /// 'cpu_attrs_'. Using a consistent value for this attribute among
+  /// hand-crafted IR and cross-compiled functions allow them to be inlined into each
+  /// other.
   static std::string target_features_attr_;
 
+  /// Mapping between CpuInfo flags and the corresponding strings.
+  /// The key is mapped to the string as follows:
+  /// CpuInfo flag -> enabled feature.
+  /// Bitwise negation of CpuInfo flag -> disabled feature.
+  const static std::map<int64_t, std::string> cpu_flag_mappings_;
+
   /// A global shared call graph for all IR functions in the main module.
   /// Used for determining dependencies when materializing IR functions.
   static CodegenCallGraph shared_call_graph_;
diff --git a/be/src/exec/filter-context.cc b/be/src/exec/filter-context.cc
index 5c39ff9..08cd666 100644
--- a/be/src/exec/filter-context.cc
+++ b/be/src/exec/filter-context.cc
@@ -388,7 +388,7 @@ Status FilterContext::CodegenInsert(LlvmCodeGen* codegen, ScalarExpr* filter_exp
 
     // Call Insert() on the bloom filter.
     llvm::Function* insert_bloom_filter_fn;
-    if (CpuInfo::IsSupported(CpuInfo::AVX2)) {
+    if (LlvmCodeGen::IsCPUFeatureEnabled(CpuInfo::AVX2)) {
       insert_bloom_filter_fn =
           codegen->GetFunction(IRFunction::BLOOM_FILTER_INSERT_AVX2, false);
     } else {
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 815e0d5..e486479 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -68,6 +68,14 @@ void WarnIfFileNotEqual(
 
 namespace impala {
 
+const int64_t CpuInfo::SSSE3;
+const int64_t CpuInfo::SSE4_1;
+const int64_t CpuInfo::SSE4_2;
+const int64_t CpuInfo::POPCNT;
+const int64_t CpuInfo::AVX;
+const int64_t CpuInfo::AVX2;
+const int64_t CpuInfo::PCLMULQDQ;
+
 bool CpuInfo::initialized_ = false;
 int64_t CpuInfo::hardware_flags_ = 0;
 int64_t CpuInfo::original_hardware_flags_;