You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/05/10 09:45:35 UTC

[GitHub] [arrow] cyb70289 opened a new pull request, #13112: ARROW-16478: Refine cpu info detection

cyb70289 opened a new pull request, #13112:
URL: https://github.com/apache/arrow/pull/13112

   This patch separates OS and ARCH depdendent code and removes CPU                                                         [24/1987]
   frequency detection (cycles_per_ms()) which is brittle and not very
   useful in practice.
   
   There are still many caveats, especially for Arm platform. It's better
   to adopt a mature library if we want more complete functionalities.
   E.g., github.com/pytorch/cpuinfo.
   
   Below are examples of cpu info detected on various platforms (some
   from virtual machines).
   
   Intel, Linux
   ------------
   Vendor: Intel
   Model: Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz
   Features (set bits):  0  1  2  3  4  5  6  7  8  9  10  11  12
   Cache sizes: 32768 1048576 37486592
   
   AMD, Linux
   ----------
   Vendor: AMD
   Model: AMD EPYC 7251 8-Core Processor
   Features (set bits):  0  1  2  3  4  5  11  12
   Cache sizes: 32768 524288 33554432
   
   Intel, MacOS
   ------------
   Vendor: Unknown
   Model: Unknown
   Features (set bits):  0  1  2  3  4
   Cache sizes: 32768 262144 12582912
   
   Intel, Windows
   --------------
   Vendor: Intel
   Model: Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz\0\0
   Features (set bits):  0  1  2  3  4  5  6  7  8  9  10  11  12
   Cache sizes: 131072 2097152 37486592
   
   Intel, MinGW
   ------------
   Vendor: Intel
   Model: Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz\0\0\0\0\0\0\0
   Features (set bits):  0  1  2  3  4  5  11  12
   Cache sizes: 131072 524288 52428800
   
   Arm, Linux
   ----------
   Vendor: Unknown
   Model: Unknown
   Features (set bits):  32
   Cache sizes: 65536 1048576 Unknown
   
   Arm, MacOS
   ----------
   Vendor: Unknown
   Model: Unknown
   Features (set bits):  32
   Cache sizes: 65536 4194304 Unknown


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #13112: ARROW-16478: Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13112:
URL: https://github.com/apache/arrow/pull/13112#issuecomment-1122168888

   https://issues.apache.org/jira/browse/ARROW-16478


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r874288921


##########
cpp/src/arrow/util/io_util_test.cc:
##########
@@ -731,5 +732,31 @@ TEST(Memory, GetRSS) {
 #endif
 }
 
+// Some loose tests to check if the cpuinfo makes sense
+TEST(CpuInfo, Basic) {
+  const CpuInfo* ci = CpuInfo::GetInstance();
+
+  ASSERT_TRUE(ci->num_cores() >= 1 && ci->num_cores() <= 1000);
+
+  const auto l1 = ci->CacheSize(CpuInfo::CacheLevel::L1);
+  const auto l2 = ci->CacheSize(CpuInfo::CacheLevel::L2);
+  const auto l3 = ci->CacheSize(CpuInfo::CacheLevel::L3);
+  ASSERT_TRUE(l1 <= l2 && l2 <= l3);
+  ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024);
+  ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024);
+  ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024);

Review Comment:
   s390x CI log shows it has 256M L3. I've increased maximal L3 size checking to 1G.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873462104


##########
cpp/src/arrow/util/io_util_test.cc:
##########
@@ -731,5 +732,31 @@ TEST(Memory, GetRSS) {
 #endif
 }
 
+// Some loose tests to check if the cpuinfo makes sense
+TEST(CpuInfo, Basic) {
+  const CpuInfo* ci = CpuInfo::GetInstance();
+
+  ASSERT_TRUE(ci->num_cores() >= 1 && ci->num_cores() <= 1000);
+
+  const auto l1 = ci->CacheSize(CpuInfo::CacheLevel::L1);
+  const auto l2 = ci->CacheSize(CpuInfo::CacheLevel::L2);
+  const auto l3 = ci->CacheSize(CpuInfo::CacheLevel::L3);
+  ASSERT_TRUE(l1 <= l2 && l2 <= l3);
+  ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024);
+  ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024);
+  ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024);
+
+  // Toggle hardware flags
+  CpuInfo* ci_rw = const_cast<CpuInfo*>(ci);
+  const int64_t original_hardware_flags = ci->hardware_flags();
+  ci_rw->EnableFeature(original_hardware_flags, false);
+  ASSERT_EQ(ci->hardware_flags(), 0);
+  ci_rw->EnableFeature(original_hardware_flags, true);
+  ASSERT_EQ(ci->hardware_flags(), original_hardware_flags);
+}
+
+// Dump detecetd cpu features in CI for manual check, not a real test
+TEST(CpuInfo, DISABLED_DebugMessage) { FAIL() << CpuInfo::GetInstance()->DebugMessage(); }

Review Comment:
   Do we need to keep this?



##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -308,262 +218,431 @@ bool RetrieveCPUInfo(int64_t* hardware_flags, std::string* model_name,
       if (features_EBX[31]) *hardware_flags |= CpuInfo::AVX512VL;
     }
   }
-
-  return true;
+}
+#elif defined(CPUINFO_ARCH_ARM)
+// Windows on Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  *hardware_flags |= CpuInfo::ASIMD;
+  // TODO: vendor, model_name
 }
 #endif
-#endif
-
-}  // namespace
 
-CpuInfo::CpuInfo()
-    : hardware_flags_(0),
-      num_cores_(1),
-      model_name_("unknown"),
-      vendor_(Vendor::Unknown) {}
-
-std::unique_ptr<CpuInfo> g_cpu_info;
-static std::once_flag cpuinfo_initialized;
-
-CpuInfo* CpuInfo::GetInstance() {
-  std::call_once(cpuinfo_initialized, []() {
-    g_cpu_info.reset(new CpuInfo);
-    g_cpu_info->Init();
-  });
-  return g_cpu_info.get();
+#elif defined(__APPLE__)
+//------------------------------ MACOS ------------------------------//
+util::optional<int64_t> IntegerSysCtlByName(const char* name) {
+  size_t len = sizeof(int64_t);
+  int64_t data = 0;
+  if (sysctlbyname(name, &data, &len, nullptr, 0) == 0) {
+    return data;
+  }
+  // ENOENT is the official errno value for non-existing sysctl's,
+  // but EINVAL and ENOTSUP have been seen in the wild.
+  if (errno != ENOENT && errno != EINVAL && errno != ENOTSUP) {
+    auto st = IOErrorFromErrno(errno, "sysctlbyname failed for '", name, "'");
+    ARROW_LOG(WARNING) << st.ToString();
+  }
+  return util::nullopt;
 }
 
-void CpuInfo::Init() {
-  std::string line;
-  std::string name;
-  std::string value;
-
-  float max_mhz = 0;
-  int num_cores = 0;
-
-  memset(&cache_sizes_, 0, sizeof(cache_sizes_));
-
-#ifdef _WIN32
-  SYSTEM_INFO system_info;
-  GetSystemInfo(&system_info);
-  num_cores = system_info.dwNumberOfProcessors;
-
-  LARGE_INTEGER performance_frequency;
-  if (QueryPerformanceFrequency(&performance_frequency)) {
-    max_mhz = static_cast<float>(performance_frequency.QuadPart);
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  static_assert(kCacheLevels >= 3, "");
+  auto c = IntegerSysCtlByName("hw.l1dcachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[0] = *c;
   }
-#elif defined(__APPLE__)
-  // On macOS, get CPU information from system information base
+  c = IntegerSysCtlByName("hw.l2cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[1] = *c;
+  }
+  c = IntegerSysCtlByName("hw.l3cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[2] = *c;
+  }
+}
+
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  // hardware_flags
   struct SysCtlCpuFeature {
     const char* name;
     int64_t flag;
   };
   std::vector<SysCtlCpuFeature> features = {
-#if defined(__aarch64__)
+#if defined(CPUINFO_ARCH_X86)
+    {"hw.optional.sse4_2",
+     CpuInfo::SSSE3 | CpuInfo::SSE4_1 | CpuInfo::SSE4_2 | CpuInfo::POPCNT},
+    {"hw.optional.avx1_0", CpuInfo::AVX},
+    {"hw.optional.avx2_0", CpuInfo::AVX2},
+    {"hw.optional.bmi1", CpuInfo::BMI1},
+    {"hw.optional.bmi2", CpuInfo::BMI2},
+    {"hw.optional.avx512f", CpuInfo::AVX512F},
+    {"hw.optional.avx512cd", CpuInfo::AVX512CD},
+    {"hw.optional.avx512dq", CpuInfo::AVX512DQ},
+    {"hw.optional.avx512bw", CpuInfo::AVX512BW},
+    {"hw.optional.avx512vl", CpuInfo::AVX512VL},
+#elif defined(CPUINFO_ARCH_ARM)
     // ARM64 (note that this is exposed under Rosetta as well)
-    {"hw.optional.neon", ASIMD},
-#else
-    // x86
-    {"hw.optional.sse4_2", SSSE3 | SSE4_1 | SSE4_2 | POPCNT},
-    {"hw.optional.avx1_0", AVX},
-    {"hw.optional.avx2_0", AVX2},
-    {"hw.optional.bmi1", BMI1},
-    {"hw.optional.bmi2", BMI2},
-    {"hw.optional.avx512f", AVX512F},
-    {"hw.optional.avx512cd", AVX512CD},
-    {"hw.optional.avx512dq", AVX512DQ},
-    {"hw.optional.avx512bw", AVX512BW},
-    {"hw.optional.avx512vl", AVX512VL},
+    {"hw.optional.neon", CpuInfo::ASIMD},
 #endif
   };
   for (const auto& feature : features) {
     auto v = IntegerSysCtlByName(feature.name);
     if (v.value_or(0)) {
-      hardware_flags_ |= feature.flag;
+      *hardware_flags |= feature.flag;
     }
   }
+
+  // TODO: vendor, model_name
+}
+
 #else
-  // Read from /proc/cpuinfo
+//------------------------------ LINUX ------------------------------//
+// Get cache size, return 0 on error
+int64_t LinuxGetCacheSize(int level) {
+  const struct {
+    int sysconf_name;
+    const char* sysfs_path;
+  } kCacheSizeEntries[] = {
+      {
+          _SC_LEVEL1_DCACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index0/size",  // l1d (index1 is l1i)
+      },
+      {
+          _SC_LEVEL2_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index2/size",  // l2
+      },
+      {
+          _SC_LEVEL3_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index3/size",  // l3
+      },
+  };
+  static_assert(sizeof(kCacheSizeEntries) / sizeof(kCacheSizeEntries[0]) == kCacheLevels,
+                "");
+
+  // get cache size by sysconf()
+  errno = 0;
+  const int64_t cache_size = sysconf(kCacheSizeEntries[level].sysconf_name);
+  if (errno == 0 && cache_size > 0) {
+    return cache_size;
+  }
+
+  // get cache size from sysfs if sysconf() fails (it does happen on Arm)
+  std::ifstream cacheinfo(kCacheSizeEntries[level].sysfs_path, std::ios::in);
+  if (!cacheinfo) {
+    return 0;
+  }
+  std::string line;

Review Comment:
   Why not use directly operator `>>`? For example:
   ```c++
   int64_t size = -1;
   char last_char = 0;
   cacheinfo >> size >> last_char;
   ```



##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -308,262 +218,431 @@ bool RetrieveCPUInfo(int64_t* hardware_flags, std::string* model_name,
       if (features_EBX[31]) *hardware_flags |= CpuInfo::AVX512VL;
     }
   }
-
-  return true;
+}
+#elif defined(CPUINFO_ARCH_ARM)
+// Windows on Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  *hardware_flags |= CpuInfo::ASIMD;
+  // TODO: vendor, model_name
 }
 #endif
-#endif
-
-}  // namespace
 
-CpuInfo::CpuInfo()
-    : hardware_flags_(0),
-      num_cores_(1),
-      model_name_("unknown"),
-      vendor_(Vendor::Unknown) {}
-
-std::unique_ptr<CpuInfo> g_cpu_info;
-static std::once_flag cpuinfo_initialized;
-
-CpuInfo* CpuInfo::GetInstance() {
-  std::call_once(cpuinfo_initialized, []() {
-    g_cpu_info.reset(new CpuInfo);
-    g_cpu_info->Init();
-  });
-  return g_cpu_info.get();
+#elif defined(__APPLE__)
+//------------------------------ MACOS ------------------------------//
+util::optional<int64_t> IntegerSysCtlByName(const char* name) {
+  size_t len = sizeof(int64_t);
+  int64_t data = 0;
+  if (sysctlbyname(name, &data, &len, nullptr, 0) == 0) {
+    return data;
+  }
+  // ENOENT is the official errno value for non-existing sysctl's,
+  // but EINVAL and ENOTSUP have been seen in the wild.
+  if (errno != ENOENT && errno != EINVAL && errno != ENOTSUP) {
+    auto st = IOErrorFromErrno(errno, "sysctlbyname failed for '", name, "'");
+    ARROW_LOG(WARNING) << st.ToString();
+  }
+  return util::nullopt;
 }
 
-void CpuInfo::Init() {
-  std::string line;
-  std::string name;
-  std::string value;
-
-  float max_mhz = 0;
-  int num_cores = 0;
-
-  memset(&cache_sizes_, 0, sizeof(cache_sizes_));
-
-#ifdef _WIN32
-  SYSTEM_INFO system_info;
-  GetSystemInfo(&system_info);
-  num_cores = system_info.dwNumberOfProcessors;
-
-  LARGE_INTEGER performance_frequency;
-  if (QueryPerformanceFrequency(&performance_frequency)) {
-    max_mhz = static_cast<float>(performance_frequency.QuadPart);
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  static_assert(kCacheLevels >= 3, "");
+  auto c = IntegerSysCtlByName("hw.l1dcachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[0] = *c;
   }
-#elif defined(__APPLE__)
-  // On macOS, get CPU information from system information base
+  c = IntegerSysCtlByName("hw.l2cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[1] = *c;
+  }
+  c = IntegerSysCtlByName("hw.l3cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[2] = *c;
+  }
+}
+
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  // hardware_flags
   struct SysCtlCpuFeature {
     const char* name;
     int64_t flag;
   };
   std::vector<SysCtlCpuFeature> features = {
-#if defined(__aarch64__)
+#if defined(CPUINFO_ARCH_X86)
+    {"hw.optional.sse4_2",
+     CpuInfo::SSSE3 | CpuInfo::SSE4_1 | CpuInfo::SSE4_2 | CpuInfo::POPCNT},
+    {"hw.optional.avx1_0", CpuInfo::AVX},
+    {"hw.optional.avx2_0", CpuInfo::AVX2},
+    {"hw.optional.bmi1", CpuInfo::BMI1},
+    {"hw.optional.bmi2", CpuInfo::BMI2},
+    {"hw.optional.avx512f", CpuInfo::AVX512F},
+    {"hw.optional.avx512cd", CpuInfo::AVX512CD},
+    {"hw.optional.avx512dq", CpuInfo::AVX512DQ},
+    {"hw.optional.avx512bw", CpuInfo::AVX512BW},
+    {"hw.optional.avx512vl", CpuInfo::AVX512VL},
+#elif defined(CPUINFO_ARCH_ARM)
     // ARM64 (note that this is exposed under Rosetta as well)
-    {"hw.optional.neon", ASIMD},
-#else
-    // x86
-    {"hw.optional.sse4_2", SSSE3 | SSE4_1 | SSE4_2 | POPCNT},
-    {"hw.optional.avx1_0", AVX},
-    {"hw.optional.avx2_0", AVX2},
-    {"hw.optional.bmi1", BMI1},
-    {"hw.optional.bmi2", BMI2},
-    {"hw.optional.avx512f", AVX512F},
-    {"hw.optional.avx512cd", AVX512CD},
-    {"hw.optional.avx512dq", AVX512DQ},
-    {"hw.optional.avx512bw", AVX512BW},
-    {"hw.optional.avx512vl", AVX512VL},
+    {"hw.optional.neon", CpuInfo::ASIMD},
 #endif
   };
   for (const auto& feature : features) {
     auto v = IntegerSysCtlByName(feature.name);
     if (v.value_or(0)) {
-      hardware_flags_ |= feature.flag;
+      *hardware_flags |= feature.flag;
     }
   }
+
+  // TODO: vendor, model_name
+}
+
 #else
-  // Read from /proc/cpuinfo
+//------------------------------ LINUX ------------------------------//
+// Get cache size, return 0 on error
+int64_t LinuxGetCacheSize(int level) {
+  const struct {
+    int sysconf_name;
+    const char* sysfs_path;
+  } kCacheSizeEntries[] = {
+      {
+          _SC_LEVEL1_DCACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index0/size",  // l1d (index1 is l1i)
+      },
+      {
+          _SC_LEVEL2_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index2/size",  // l2
+      },
+      {
+          _SC_LEVEL3_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index3/size",  // l3
+      },
+  };
+  static_assert(sizeof(kCacheSizeEntries) / sizeof(kCacheSizeEntries[0]) == kCacheLevels,
+                "");
+
+  // get cache size by sysconf()
+  errno = 0;
+  const int64_t cache_size = sysconf(kCacheSizeEntries[level].sysconf_name);
+  if (errno == 0 && cache_size > 0) {
+    return cache_size;
+  }
+
+  // get cache size from sysfs if sysconf() fails (it does happen on Arm)
+  std::ifstream cacheinfo(kCacheSizeEntries[level].sysfs_path, std::ios::in);
+  if (!cacheinfo) {
+    return 0;
+  }
+  std::string line;
+  std::getline(cacheinfo, line);
+  if (line.empty()) {
+    return 0;
+  }
+  // line: 65536, 64K, 1M, etc.
+  char* last_char;
+  errno = 0;
+  auto size = std::strtoull(line.c_str(), &last_char, 0);
+  if (errno != 0) {
+    return 0;
+  }
+  const int unit = std::toupper(static_cast<unsigned char>(*last_char));
+  if (unit == 'K') {
+    size <<= 10;
+  } else if (unit == 'M') {
+    size <<= 20;
+  } else if (unit == 'G') {
+    size <<= 30;
+  } else {
+    return 0;
+  }
+  return static_cast<int64_t>(size);
+}
+
+// Helper function to parse for hardware flags from /proc/cpuinfo
+// values contains a list of space-separated flags.  check to see if the flags we
+// care about are present.
+// Returns a bitmap of flags.
+int64_t LinuxParseCpuFlags(const std::string& values) {
+  const struct {
+    std::string name;
+    int64_t flag;
+  } flag_mappings[] = {
+#if defined(CPUINFO_ARCH_X86)
+    {"ssse3", CpuInfo::SSSE3},
+    {"sse4_1", CpuInfo::SSE4_1},
+    {"sse4_2", CpuInfo::SSE4_2},
+    {"popcnt", CpuInfo::POPCNT},
+    {"avx", CpuInfo::AVX},
+    {"avx2", CpuInfo::AVX2},
+    {"avx512f", CpuInfo::AVX512F},
+    {"avx512cd", CpuInfo::AVX512CD},
+    {"avx512vl", CpuInfo::AVX512VL},
+    {"avx512dq", CpuInfo::AVX512DQ},
+    {"avx512bw", CpuInfo::AVX512BW},
+    {"bmi1", CpuInfo::BMI1},
+    {"bmi2", CpuInfo::BMI2},
+#elif defined(CPUINFO_ARCH_ARM)
+    {"asimd", CpuInfo::ASIMD},
+#endif
+  };
+  const int64_t num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
+
+  int64_t flags = 0;
+  for (int i = 0; i < num_flags; ++i) {
+    if (values.find(flag_mappings[i].name) != std::string::npos) {
+      flags |= flag_mappings[i].flag;
+    }
+  }
+  return flags;
+}
+
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  for (int i = 0; i < kCacheLevels; ++i) {
+    const int64_t cache_size = LinuxGetCacheSize(i);
+    if (cache_size > 0) {
+      (*cache_sizes)[i] = cache_size;
+    }
+  }
+}
+
+// Read from /proc/cpuinfo
+// TODO: vendor, model_name for Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
   std::ifstream cpuinfo("/proc/cpuinfo", std::ios::in);
   while (cpuinfo) {
+    std::string line;
     std::getline(cpuinfo, line);
-    size_t colon = line.find(':');
+    const size_t colon = line.find(':');
     if (colon != std::string::npos) {
-      name = TrimString(line.substr(0, colon - 1));
-      value = TrimString(line.substr(colon + 1, std::string::npos));
+      const std::string name = TrimString(line.substr(0, colon - 1));
+      const std::string value = TrimString(line.substr(colon + 1, std::string::npos));
       if (name.compare("flags") == 0 || name.compare("Features") == 0) {
-        hardware_flags_ |= ParseCPUFlags(value);
-      } else if (name.compare("cpu MHz") == 0) {
-        // Every core will report a different speed.  We'll take the max, assuming
-        // that when impala is running, the core will not be in a lower power state.
-        // TODO: is there a more robust way to do this, such as
-        // Window's QueryPerformanceFrequency()
-        float mhz = static_cast<float>(atof(value.c_str()));
-        max_mhz = max(mhz, max_mhz);
-      } else if (name.compare("processor") == 0) {
-        ++num_cores;
+        *hardware_flags |= LinuxParseCpuFlags(value);
       } else if (name.compare("model name") == 0) {
-        model_name_ = value;
+        *model_name = value;
       } else if (name.compare("vendor_id") == 0) {
         if (value.compare("GenuineIntel") == 0) {
-          vendor_ = Vendor::Intel;
+          *vendor = CpuInfo::Vendor::Intel;
         } else if (value.compare("AuthenticAMD") == 0) {
-          vendor_ = Vendor::AMD;
+          *vendor = CpuInfo::Vendor::AMD;
         }
       }
     }
   }
-  if (cpuinfo.is_open()) cpuinfo.close();
-#endif
+}
+#endif  // WINDOWS, MACOS, LINUX
+
+//============================== Arch Dependent ==============================//
+
+#if defined(CPUINFO_ARCH_X86)
+//------------------------------ X86_64 ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  enum {
+    USER_SIMD_NONE,
+    USER_SIMD_SSE4_2,
+    USER_SIMD_AVX,
+    USER_SIMD_AVX2,
+    USER_SIMD_AVX512,
+    USER_SIMD_MAX,
+  };
 
-#ifdef __APPLE__
-  // On macOS, get cache size from system information base
-  SetDefaultCacheSize();
-  auto c = IntegerSysCtlByName("hw.l1dcachesize");
-  if (c.has_value()) {
-    cache_sizes_[0] = *c;
+  int level = USER_SIMD_MAX;
+  // Parse the level
+  if (simd_level == "AVX512") {
+    level = USER_SIMD_AVX512;
+  } else if (simd_level == "AVX2") {
+    level = USER_SIMD_AVX2;
+  } else if (simd_level == "AVX") {
+    level = USER_SIMD_AVX;
+  } else if (simd_level == "SSE4_2") {
+    level = USER_SIMD_SSE4_2;
+  } else if (simd_level == "NONE") {
+    level = USER_SIMD_NONE;
+  } else {
+    return false;
   }
-  c = IntegerSysCtlByName("hw.l2cachesize");
-  if (c.has_value()) {
-    cache_sizes_[1] = *c;
+
+  // Disable feature as the level
+  if (level < USER_SIMD_AVX512) {
+    *hardware_flags &= ~CpuInfo::AVX512;
   }
-  c = IntegerSysCtlByName("hw.l3cachesize");
-  if (c.has_value()) {
-    cache_sizes_[2] = *c;
+  if (level < USER_SIMD_AVX2) {
+    *hardware_flags &= ~(CpuInfo::AVX2 | CpuInfo::BMI2);
   }
-#elif _WIN32
-  if (!RetrieveCacheSize(cache_sizes_)) {
-    SetDefaultCacheSize();
+  if (level < USER_SIMD_AVX) {
+    *hardware_flags &= ~CpuInfo::AVX;
   }
-#ifndef _M_ARM64
-  RetrieveCPUInfo(&hardware_flags_, &model_name_, &vendor_);
-#endif
-#else
-  SetDefaultCacheSize();
-#endif
-
-  if (max_mhz != 0) {
-    cycles_per_ms_ = static_cast<int64_t>(max_mhz);
-#ifndef _WIN32
-    cycles_per_ms_ *= 1000;
-#endif
-  } else {
-    cycles_per_ms_ = 1000000;
+  if (level < USER_SIMD_SSE4_2) {
+    *hardware_flags &= ~(CpuInfo::SSE4_2 | CpuInfo::BMI1);
   }
-  original_hardware_flags_ = hardware_flags_;
+  return true;
+}
 
-  if (num_cores > 0) {
-    num_cores_ = num_cores;
-  } else {
-    num_cores_ = 1;
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {
+#if defined(ARROW_HAVE_SSE4_2)
+  if (!ci->IsDetected(CpuInfo::SSE4_2)) {
+    DCHECK(false) << "CPU does not support the Supplemental SSE4_2 instruction set";
   }
-
-  // Parse the user simd level
-  ParseUserSimdLevel();
+#endif
 }
 
-void CpuInfo::VerifyCpuRequirements() {
-#ifdef ARROW_HAVE_SSE4_2
-  if (!IsSupported(CpuInfo::SSSE3)) {
-    DCHECK(false) << "CPU does not support the Supplemental SSE3 instruction set";
+#elif defined(CPUINFO_ARCH_ARM)
+//------------------------------ AARCH64 ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  if (simd_level == "NONE") {
+    *hardware_flags &= ~CpuInfo::ASIMD;
+    return true;
   }
-#endif
-#if defined(ARROW_HAVE_NEON)
-  if (!IsSupported(CpuInfo::ASIMD)) {
+  return false;
+}
+
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {
+  if (!ci->IsDetected(CpuInfo::ASIMD)) {
     DCHECK(false) << "CPU does not support the Armv8 Neon instruction set";
   }
-#endif
 }
 
-bool CpuInfo::CanUseSSE4_2() const {
-#if defined(ARROW_HAVE_SSE4_2)
-  return IsSupported(CpuInfo::SSE4_2);
 #else
-  return false;
-#endif
+//------------------------------ PPC, ... ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  return true;
 }
 
-void CpuInfo::EnableFeature(int64_t flag, bool enable) {
-  if (!enable) {
-    hardware_flags_ &= ~flag;
-  } else {
-    // Can't turn something on that can't be supported
-    DCHECK_NE(original_hardware_flags_ & flag, 0);
-    hardware_flags_ |= flag;
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {}
+
+#endif  // X86, ARM, PPC
+
+}  // namespace
+
+struct CpuInfo::Impl {
+  int64_t hardware_flags = 0;
+  int num_cores = 0;
+  int64_t original_hardware_flags = 0;
+  Vendor vendor = Vendor::Unknown;
+  std::string model_name = "Unknown";
+  std::array<int64_t, kCacheLevels> cache_sizes{};
+
+  Impl() {
+    OsRetrieveCacheSize(&cache_sizes);
+    OsRetrieveCpuInfo(&hardware_flags, &vendor, &model_name);
+    original_hardware_flags = hardware_flags;
+    num_cores = std::max(static_cast<int>(std::thread::hardware_concurrency()), 1);
+
+    // parse user simd level
+    auto maybe_env_var = GetEnvVar("ARROW_USER_SIMD_LEVEL");
+    if (!maybe_env_var.ok()) {
+      return;
+    }
+    std::string& s = maybe_env_var.ValueUnsafe();

Review Comment:
   ```suggestion
       std::string s = *std::move(maybe_env_var);
   ```



##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -200,44 +93,59 @@ bool RetrieveCacheSize(int64_t* cache_sizes) {
           GetModuleHandle("kernel32"), "GetLogicalProcessorInformation");
 
   if (!func_pointer) {
-    return false;
+    return;

Review Comment:
   Can we log warnings in case of unexpected errors in this function?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873588128


##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -308,262 +218,431 @@ bool RetrieveCPUInfo(int64_t* hardware_flags, std::string* model_name,
       if (features_EBX[31]) *hardware_flags |= CpuInfo::AVX512VL;
     }
   }
-
-  return true;
+}
+#elif defined(CPUINFO_ARCH_ARM)
+// Windows on Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  *hardware_flags |= CpuInfo::ASIMD;
+  // TODO: vendor, model_name
 }
 #endif
-#endif
-
-}  // namespace
 
-CpuInfo::CpuInfo()
-    : hardware_flags_(0),
-      num_cores_(1),
-      model_name_("unknown"),
-      vendor_(Vendor::Unknown) {}
-
-std::unique_ptr<CpuInfo> g_cpu_info;
-static std::once_flag cpuinfo_initialized;
-
-CpuInfo* CpuInfo::GetInstance() {
-  std::call_once(cpuinfo_initialized, []() {
-    g_cpu_info.reset(new CpuInfo);
-    g_cpu_info->Init();
-  });
-  return g_cpu_info.get();
+#elif defined(__APPLE__)
+//------------------------------ MACOS ------------------------------//
+util::optional<int64_t> IntegerSysCtlByName(const char* name) {
+  size_t len = sizeof(int64_t);
+  int64_t data = 0;
+  if (sysctlbyname(name, &data, &len, nullptr, 0) == 0) {
+    return data;
+  }
+  // ENOENT is the official errno value for non-existing sysctl's,
+  // but EINVAL and ENOTSUP have been seen in the wild.
+  if (errno != ENOENT && errno != EINVAL && errno != ENOTSUP) {
+    auto st = IOErrorFromErrno(errno, "sysctlbyname failed for '", name, "'");
+    ARROW_LOG(WARNING) << st.ToString();
+  }
+  return util::nullopt;
 }
 
-void CpuInfo::Init() {
-  std::string line;
-  std::string name;
-  std::string value;
-
-  float max_mhz = 0;
-  int num_cores = 0;
-
-  memset(&cache_sizes_, 0, sizeof(cache_sizes_));
-
-#ifdef _WIN32
-  SYSTEM_INFO system_info;
-  GetSystemInfo(&system_info);
-  num_cores = system_info.dwNumberOfProcessors;
-
-  LARGE_INTEGER performance_frequency;
-  if (QueryPerformanceFrequency(&performance_frequency)) {
-    max_mhz = static_cast<float>(performance_frequency.QuadPart);
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  static_assert(kCacheLevels >= 3, "");
+  auto c = IntegerSysCtlByName("hw.l1dcachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[0] = *c;
   }
-#elif defined(__APPLE__)
-  // On macOS, get CPU information from system information base
+  c = IntegerSysCtlByName("hw.l2cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[1] = *c;
+  }
+  c = IntegerSysCtlByName("hw.l3cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[2] = *c;
+  }
+}
+
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  // hardware_flags
   struct SysCtlCpuFeature {
     const char* name;
     int64_t flag;
   };
   std::vector<SysCtlCpuFeature> features = {
-#if defined(__aarch64__)
+#if defined(CPUINFO_ARCH_X86)
+    {"hw.optional.sse4_2",
+     CpuInfo::SSSE3 | CpuInfo::SSE4_1 | CpuInfo::SSE4_2 | CpuInfo::POPCNT},
+    {"hw.optional.avx1_0", CpuInfo::AVX},
+    {"hw.optional.avx2_0", CpuInfo::AVX2},
+    {"hw.optional.bmi1", CpuInfo::BMI1},
+    {"hw.optional.bmi2", CpuInfo::BMI2},
+    {"hw.optional.avx512f", CpuInfo::AVX512F},
+    {"hw.optional.avx512cd", CpuInfo::AVX512CD},
+    {"hw.optional.avx512dq", CpuInfo::AVX512DQ},
+    {"hw.optional.avx512bw", CpuInfo::AVX512BW},
+    {"hw.optional.avx512vl", CpuInfo::AVX512VL},
+#elif defined(CPUINFO_ARCH_ARM)
     // ARM64 (note that this is exposed under Rosetta as well)
-    {"hw.optional.neon", ASIMD},
-#else
-    // x86
-    {"hw.optional.sse4_2", SSSE3 | SSE4_1 | SSE4_2 | POPCNT},
-    {"hw.optional.avx1_0", AVX},
-    {"hw.optional.avx2_0", AVX2},
-    {"hw.optional.bmi1", BMI1},
-    {"hw.optional.bmi2", BMI2},
-    {"hw.optional.avx512f", AVX512F},
-    {"hw.optional.avx512cd", AVX512CD},
-    {"hw.optional.avx512dq", AVX512DQ},
-    {"hw.optional.avx512bw", AVX512BW},
-    {"hw.optional.avx512vl", AVX512VL},
+    {"hw.optional.neon", CpuInfo::ASIMD},
 #endif
   };
   for (const auto& feature : features) {
     auto v = IntegerSysCtlByName(feature.name);
     if (v.value_or(0)) {
-      hardware_flags_ |= feature.flag;
+      *hardware_flags |= feature.flag;
     }
   }
+
+  // TODO: vendor, model_name
+}
+
 #else
-  // Read from /proc/cpuinfo
+//------------------------------ LINUX ------------------------------//
+// Get cache size, return 0 on error
+int64_t LinuxGetCacheSize(int level) {
+  const struct {
+    int sysconf_name;
+    const char* sysfs_path;
+  } kCacheSizeEntries[] = {
+      {
+          _SC_LEVEL1_DCACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index0/size",  // l1d (index1 is l1i)
+      },
+      {
+          _SC_LEVEL2_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index2/size",  // l2
+      },
+      {
+          _SC_LEVEL3_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index3/size",  // l3
+      },
+  };
+  static_assert(sizeof(kCacheSizeEntries) / sizeof(kCacheSizeEntries[0]) == kCacheLevels,
+                "");
+
+  // get cache size by sysconf()
+  errno = 0;
+  const int64_t cache_size = sysconf(kCacheSizeEntries[level].sysconf_name);
+  if (errno == 0 && cache_size > 0) {
+    return cache_size;
+  }
+
+  // get cache size from sysfs if sysconf() fails (it does happen on Arm)
+  std::ifstream cacheinfo(kCacheSizeEntries[level].sysfs_path, std::ios::in);
+  if (!cacheinfo) {
+    return 0;
+  }
+  std::string line;

Review Comment:
   Good catch, code updated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873588423


##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -308,262 +218,431 @@ bool RetrieveCPUInfo(int64_t* hardware_flags, std::string* model_name,
       if (features_EBX[31]) *hardware_flags |= CpuInfo::AVX512VL;
     }
   }
-
-  return true;
+}
+#elif defined(CPUINFO_ARCH_ARM)
+// Windows on Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  *hardware_flags |= CpuInfo::ASIMD;
+  // TODO: vendor, model_name
 }
 #endif
-#endif
-
-}  // namespace
 
-CpuInfo::CpuInfo()
-    : hardware_flags_(0),
-      num_cores_(1),
-      model_name_("unknown"),
-      vendor_(Vendor::Unknown) {}
-
-std::unique_ptr<CpuInfo> g_cpu_info;
-static std::once_flag cpuinfo_initialized;
-
-CpuInfo* CpuInfo::GetInstance() {
-  std::call_once(cpuinfo_initialized, []() {
-    g_cpu_info.reset(new CpuInfo);
-    g_cpu_info->Init();
-  });
-  return g_cpu_info.get();
+#elif defined(__APPLE__)
+//------------------------------ MACOS ------------------------------//
+util::optional<int64_t> IntegerSysCtlByName(const char* name) {
+  size_t len = sizeof(int64_t);
+  int64_t data = 0;
+  if (sysctlbyname(name, &data, &len, nullptr, 0) == 0) {
+    return data;
+  }
+  // ENOENT is the official errno value for non-existing sysctl's,
+  // but EINVAL and ENOTSUP have been seen in the wild.
+  if (errno != ENOENT && errno != EINVAL && errno != ENOTSUP) {
+    auto st = IOErrorFromErrno(errno, "sysctlbyname failed for '", name, "'");
+    ARROW_LOG(WARNING) << st.ToString();
+  }
+  return util::nullopt;
 }
 
-void CpuInfo::Init() {
-  std::string line;
-  std::string name;
-  std::string value;
-
-  float max_mhz = 0;
-  int num_cores = 0;
-
-  memset(&cache_sizes_, 0, sizeof(cache_sizes_));
-
-#ifdef _WIN32
-  SYSTEM_INFO system_info;
-  GetSystemInfo(&system_info);
-  num_cores = system_info.dwNumberOfProcessors;
-
-  LARGE_INTEGER performance_frequency;
-  if (QueryPerformanceFrequency(&performance_frequency)) {
-    max_mhz = static_cast<float>(performance_frequency.QuadPart);
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  static_assert(kCacheLevels >= 3, "");
+  auto c = IntegerSysCtlByName("hw.l1dcachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[0] = *c;
   }
-#elif defined(__APPLE__)
-  // On macOS, get CPU information from system information base
+  c = IntegerSysCtlByName("hw.l2cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[1] = *c;
+  }
+  c = IntegerSysCtlByName("hw.l3cachesize");
+  if (c.has_value()) {
+    (*cache_sizes)[2] = *c;
+  }
+}
+
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
+  // hardware_flags
   struct SysCtlCpuFeature {
     const char* name;
     int64_t flag;
   };
   std::vector<SysCtlCpuFeature> features = {
-#if defined(__aarch64__)
+#if defined(CPUINFO_ARCH_X86)
+    {"hw.optional.sse4_2",
+     CpuInfo::SSSE3 | CpuInfo::SSE4_1 | CpuInfo::SSE4_2 | CpuInfo::POPCNT},
+    {"hw.optional.avx1_0", CpuInfo::AVX},
+    {"hw.optional.avx2_0", CpuInfo::AVX2},
+    {"hw.optional.bmi1", CpuInfo::BMI1},
+    {"hw.optional.bmi2", CpuInfo::BMI2},
+    {"hw.optional.avx512f", CpuInfo::AVX512F},
+    {"hw.optional.avx512cd", CpuInfo::AVX512CD},
+    {"hw.optional.avx512dq", CpuInfo::AVX512DQ},
+    {"hw.optional.avx512bw", CpuInfo::AVX512BW},
+    {"hw.optional.avx512vl", CpuInfo::AVX512VL},
+#elif defined(CPUINFO_ARCH_ARM)
     // ARM64 (note that this is exposed under Rosetta as well)
-    {"hw.optional.neon", ASIMD},
-#else
-    // x86
-    {"hw.optional.sse4_2", SSSE3 | SSE4_1 | SSE4_2 | POPCNT},
-    {"hw.optional.avx1_0", AVX},
-    {"hw.optional.avx2_0", AVX2},
-    {"hw.optional.bmi1", BMI1},
-    {"hw.optional.bmi2", BMI2},
-    {"hw.optional.avx512f", AVX512F},
-    {"hw.optional.avx512cd", AVX512CD},
-    {"hw.optional.avx512dq", AVX512DQ},
-    {"hw.optional.avx512bw", AVX512BW},
-    {"hw.optional.avx512vl", AVX512VL},
+    {"hw.optional.neon", CpuInfo::ASIMD},
 #endif
   };
   for (const auto& feature : features) {
     auto v = IntegerSysCtlByName(feature.name);
     if (v.value_or(0)) {
-      hardware_flags_ |= feature.flag;
+      *hardware_flags |= feature.flag;
     }
   }
+
+  // TODO: vendor, model_name
+}
+
 #else
-  // Read from /proc/cpuinfo
+//------------------------------ LINUX ------------------------------//
+// Get cache size, return 0 on error
+int64_t LinuxGetCacheSize(int level) {
+  const struct {
+    int sysconf_name;
+    const char* sysfs_path;
+  } kCacheSizeEntries[] = {
+      {
+          _SC_LEVEL1_DCACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index0/size",  // l1d (index1 is l1i)
+      },
+      {
+          _SC_LEVEL2_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index2/size",  // l2
+      },
+      {
+          _SC_LEVEL3_CACHE_SIZE,
+          "/sys/devices/system/cpu/cpu0/cache/index3/size",  // l3
+      },
+  };
+  static_assert(sizeof(kCacheSizeEntries) / sizeof(kCacheSizeEntries[0]) == kCacheLevels,
+                "");
+
+  // get cache size by sysconf()
+  errno = 0;
+  const int64_t cache_size = sysconf(kCacheSizeEntries[level].sysconf_name);
+  if (errno == 0 && cache_size > 0) {
+    return cache_size;
+  }
+
+  // get cache size from sysfs if sysconf() fails (it does happen on Arm)
+  std::ifstream cacheinfo(kCacheSizeEntries[level].sysfs_path, std::ios::in);
+  if (!cacheinfo) {
+    return 0;
+  }
+  std::string line;
+  std::getline(cacheinfo, line);
+  if (line.empty()) {
+    return 0;
+  }
+  // line: 65536, 64K, 1M, etc.
+  char* last_char;
+  errno = 0;
+  auto size = std::strtoull(line.c_str(), &last_char, 0);
+  if (errno != 0) {
+    return 0;
+  }
+  const int unit = std::toupper(static_cast<unsigned char>(*last_char));
+  if (unit == 'K') {
+    size <<= 10;
+  } else if (unit == 'M') {
+    size <<= 20;
+  } else if (unit == 'G') {
+    size <<= 30;
+  } else {
+    return 0;
+  }
+  return static_cast<int64_t>(size);
+}
+
+// Helper function to parse for hardware flags from /proc/cpuinfo
+// values contains a list of space-separated flags.  check to see if the flags we
+// care about are present.
+// Returns a bitmap of flags.
+int64_t LinuxParseCpuFlags(const std::string& values) {
+  const struct {
+    std::string name;
+    int64_t flag;
+  } flag_mappings[] = {
+#if defined(CPUINFO_ARCH_X86)
+    {"ssse3", CpuInfo::SSSE3},
+    {"sse4_1", CpuInfo::SSE4_1},
+    {"sse4_2", CpuInfo::SSE4_2},
+    {"popcnt", CpuInfo::POPCNT},
+    {"avx", CpuInfo::AVX},
+    {"avx2", CpuInfo::AVX2},
+    {"avx512f", CpuInfo::AVX512F},
+    {"avx512cd", CpuInfo::AVX512CD},
+    {"avx512vl", CpuInfo::AVX512VL},
+    {"avx512dq", CpuInfo::AVX512DQ},
+    {"avx512bw", CpuInfo::AVX512BW},
+    {"bmi1", CpuInfo::BMI1},
+    {"bmi2", CpuInfo::BMI2},
+#elif defined(CPUINFO_ARCH_ARM)
+    {"asimd", CpuInfo::ASIMD},
+#endif
+  };
+  const int64_t num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
+
+  int64_t flags = 0;
+  for (int i = 0; i < num_flags; ++i) {
+    if (values.find(flag_mappings[i].name) != std::string::npos) {
+      flags |= flag_mappings[i].flag;
+    }
+  }
+  return flags;
+}
+
+void OsRetrieveCacheSize(std::array<int64_t, kCacheLevels>* cache_sizes) {
+  for (int i = 0; i < kCacheLevels; ++i) {
+    const int64_t cache_size = LinuxGetCacheSize(i);
+    if (cache_size > 0) {
+      (*cache_sizes)[i] = cache_size;
+    }
+  }
+}
+
+// Read from /proc/cpuinfo
+// TODO: vendor, model_name for Arm
+void OsRetrieveCpuInfo(int64_t* hardware_flags, CpuInfo::Vendor* vendor,
+                       std::string* model_name) {
   std::ifstream cpuinfo("/proc/cpuinfo", std::ios::in);
   while (cpuinfo) {
+    std::string line;
     std::getline(cpuinfo, line);
-    size_t colon = line.find(':');
+    const size_t colon = line.find(':');
     if (colon != std::string::npos) {
-      name = TrimString(line.substr(0, colon - 1));
-      value = TrimString(line.substr(colon + 1, std::string::npos));
+      const std::string name = TrimString(line.substr(0, colon - 1));
+      const std::string value = TrimString(line.substr(colon + 1, std::string::npos));
       if (name.compare("flags") == 0 || name.compare("Features") == 0) {
-        hardware_flags_ |= ParseCPUFlags(value);
-      } else if (name.compare("cpu MHz") == 0) {
-        // Every core will report a different speed.  We'll take the max, assuming
-        // that when impala is running, the core will not be in a lower power state.
-        // TODO: is there a more robust way to do this, such as
-        // Window's QueryPerformanceFrequency()
-        float mhz = static_cast<float>(atof(value.c_str()));
-        max_mhz = max(mhz, max_mhz);
-      } else if (name.compare("processor") == 0) {
-        ++num_cores;
+        *hardware_flags |= LinuxParseCpuFlags(value);
       } else if (name.compare("model name") == 0) {
-        model_name_ = value;
+        *model_name = value;
       } else if (name.compare("vendor_id") == 0) {
         if (value.compare("GenuineIntel") == 0) {
-          vendor_ = Vendor::Intel;
+          *vendor = CpuInfo::Vendor::Intel;
         } else if (value.compare("AuthenticAMD") == 0) {
-          vendor_ = Vendor::AMD;
+          *vendor = CpuInfo::Vendor::AMD;
         }
       }
     }
   }
-  if (cpuinfo.is_open()) cpuinfo.close();
-#endif
+}
+#endif  // WINDOWS, MACOS, LINUX
+
+//============================== Arch Dependent ==============================//
+
+#if defined(CPUINFO_ARCH_X86)
+//------------------------------ X86_64 ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  enum {
+    USER_SIMD_NONE,
+    USER_SIMD_SSE4_2,
+    USER_SIMD_AVX,
+    USER_SIMD_AVX2,
+    USER_SIMD_AVX512,
+    USER_SIMD_MAX,
+  };
 
-#ifdef __APPLE__
-  // On macOS, get cache size from system information base
-  SetDefaultCacheSize();
-  auto c = IntegerSysCtlByName("hw.l1dcachesize");
-  if (c.has_value()) {
-    cache_sizes_[0] = *c;
+  int level = USER_SIMD_MAX;
+  // Parse the level
+  if (simd_level == "AVX512") {
+    level = USER_SIMD_AVX512;
+  } else if (simd_level == "AVX2") {
+    level = USER_SIMD_AVX2;
+  } else if (simd_level == "AVX") {
+    level = USER_SIMD_AVX;
+  } else if (simd_level == "SSE4_2") {
+    level = USER_SIMD_SSE4_2;
+  } else if (simd_level == "NONE") {
+    level = USER_SIMD_NONE;
+  } else {
+    return false;
   }
-  c = IntegerSysCtlByName("hw.l2cachesize");
-  if (c.has_value()) {
-    cache_sizes_[1] = *c;
+
+  // Disable feature as the level
+  if (level < USER_SIMD_AVX512) {
+    *hardware_flags &= ~CpuInfo::AVX512;
   }
-  c = IntegerSysCtlByName("hw.l3cachesize");
-  if (c.has_value()) {
-    cache_sizes_[2] = *c;
+  if (level < USER_SIMD_AVX2) {
+    *hardware_flags &= ~(CpuInfo::AVX2 | CpuInfo::BMI2);
   }
-#elif _WIN32
-  if (!RetrieveCacheSize(cache_sizes_)) {
-    SetDefaultCacheSize();
+  if (level < USER_SIMD_AVX) {
+    *hardware_flags &= ~CpuInfo::AVX;
   }
-#ifndef _M_ARM64
-  RetrieveCPUInfo(&hardware_flags_, &model_name_, &vendor_);
-#endif
-#else
-  SetDefaultCacheSize();
-#endif
-
-  if (max_mhz != 0) {
-    cycles_per_ms_ = static_cast<int64_t>(max_mhz);
-#ifndef _WIN32
-    cycles_per_ms_ *= 1000;
-#endif
-  } else {
-    cycles_per_ms_ = 1000000;
+  if (level < USER_SIMD_SSE4_2) {
+    *hardware_flags &= ~(CpuInfo::SSE4_2 | CpuInfo::BMI1);
   }
-  original_hardware_flags_ = hardware_flags_;
+  return true;
+}
 
-  if (num_cores > 0) {
-    num_cores_ = num_cores;
-  } else {
-    num_cores_ = 1;
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {
+#if defined(ARROW_HAVE_SSE4_2)
+  if (!ci->IsDetected(CpuInfo::SSE4_2)) {
+    DCHECK(false) << "CPU does not support the Supplemental SSE4_2 instruction set";
   }
-
-  // Parse the user simd level
-  ParseUserSimdLevel();
+#endif
 }
 
-void CpuInfo::VerifyCpuRequirements() {
-#ifdef ARROW_HAVE_SSE4_2
-  if (!IsSupported(CpuInfo::SSSE3)) {
-    DCHECK(false) << "CPU does not support the Supplemental SSE3 instruction set";
+#elif defined(CPUINFO_ARCH_ARM)
+//------------------------------ AARCH64 ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  if (simd_level == "NONE") {
+    *hardware_flags &= ~CpuInfo::ASIMD;
+    return true;
   }
-#endif
-#if defined(ARROW_HAVE_NEON)
-  if (!IsSupported(CpuInfo::ASIMD)) {
+  return false;
+}
+
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {
+  if (!ci->IsDetected(CpuInfo::ASIMD)) {
     DCHECK(false) << "CPU does not support the Armv8 Neon instruction set";
   }
-#endif
 }
 
-bool CpuInfo::CanUseSSE4_2() const {
-#if defined(ARROW_HAVE_SSE4_2)
-  return IsSupported(CpuInfo::SSE4_2);
 #else
-  return false;
-#endif
+//------------------------------ PPC, ... ------------------------------//
+bool ArchParseUserSimdLevel(const std::string& simd_level, int64_t* hardware_flags) {
+  return true;
 }
 
-void CpuInfo::EnableFeature(int64_t flag, bool enable) {
-  if (!enable) {
-    hardware_flags_ &= ~flag;
-  } else {
-    // Can't turn something on that can't be supported
-    DCHECK_NE(original_hardware_flags_ & flag, 0);
-    hardware_flags_ |= flag;
+void ArchVerifyCpuRequirements(const CpuInfo* ci) {}
+
+#endif  // X86, ARM, PPC
+
+}  // namespace
+
+struct CpuInfo::Impl {
+  int64_t hardware_flags = 0;
+  int num_cores = 0;
+  int64_t original_hardware_flags = 0;
+  Vendor vendor = Vendor::Unknown;
+  std::string model_name = "Unknown";
+  std::array<int64_t, kCacheLevels> cache_sizes{};
+
+  Impl() {
+    OsRetrieveCacheSize(&cache_sizes);
+    OsRetrieveCpuInfo(&hardware_flags, &vendor, &model_name);
+    original_hardware_flags = hardware_flags;
+    num_cores = std::max(static_cast<int>(std::thread::hardware_concurrency()), 1);
+
+    // parse user simd level
+    auto maybe_env_var = GetEnvVar("ARROW_USER_SIMD_LEVEL");
+    if (!maybe_env_var.ok()) {
+      return;
+    }
+    std::string& s = maybe_env_var.ValueUnsafe();

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873461937


##########
cpp/src/parquet/level_conversion.cc:
##########
@@ -137,7 +137,9 @@ void DefLevelsToBitmap(const int16_t* def_levels, int64_t num_def_levels,
   // is deleted in a follow-up release.
   if (level_info.rep_level > 0) {
 #if defined(ARROW_HAVE_RUNTIME_BMI2)
-    if (CpuInfo::GetInstance()->HasEfficientBmi2()) {
+    const CpuInfo* ci = CpuInfo::GetInstance();
+    // BMI2 (pext, pdep) is only efficient on Intel X86 processors.

Review Comment:
   I would rather keep something like the `HasEfficientBmi2` abstraction because this may become more complicated in the future: https://issues.apache.org/jira/browse/ARROW-10548



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #13112: ARROW-16478: Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13112:
URL: https://github.com/apache/arrow/pull/13112#issuecomment-1122168910

   :warning: Ticket **has not been started in JIRA**, please click 'Start Progress'.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 closed pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 closed pull request #13112: ARROW-16478: [C++] Refine cpu info detection
URL: https://github.com/apache/arrow/pull/13112


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873486038


##########
cpp/src/arrow/util/io_util_test.cc:
##########
@@ -731,5 +732,31 @@ TEST(Memory, GetRSS) {
 #endif
 }
 
+// Some loose tests to check if the cpuinfo makes sense
+TEST(CpuInfo, Basic) {
+  const CpuInfo* ci = CpuInfo::GetInstance();
+
+  ASSERT_TRUE(ci->num_cores() >= 1 && ci->num_cores() <= 1000);
+
+  const auto l1 = ci->CacheSize(CpuInfo::CacheLevel::L1);
+  const auto l2 = ci->CacheSize(CpuInfo::CacheLevel::L2);
+  const auto l3 = ci->CacheSize(CpuInfo::CacheLevel::L3);
+  ASSERT_TRUE(l1 <= l2 && l2 <= l3);
+  ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024);
+  ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024);
+  ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024);

Review Comment:
   Make diagnosing easier:
   ```suggestion
     ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024) << "unexpected L1 size: " << l1;
     ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024) << "unexpected L2 size: " << l2;
     ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024) << "unexpected L3 size: " << l3;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873588777


##########
cpp/src/arrow/util/io_util_test.cc:
##########
@@ -731,5 +732,31 @@ TEST(Memory, GetRSS) {
 #endif
 }
 
+// Some loose tests to check if the cpuinfo makes sense
+TEST(CpuInfo, Basic) {
+  const CpuInfo* ci = CpuInfo::GetInstance();
+
+  ASSERT_TRUE(ci->num_cores() >= 1 && ci->num_cores() <= 1000);
+
+  const auto l1 = ci->CacheSize(CpuInfo::CacheLevel::L1);
+  const auto l2 = ci->CacheSize(CpuInfo::CacheLevel::L2);
+  const auto l3 = ci->CacheSize(CpuInfo::CacheLevel::L3);
+  ASSERT_TRUE(l1 <= l2 && l2 <= l3);
+  ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024);
+  ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024);
+  ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024);

Review Comment:
   Done. Waiting for CI error log...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] ursabot commented on pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #13112:
URL: https://github.com/apache/arrow/pull/13112#issuecomment-1129796644

   Benchmark runs are scheduled for baseline = ed084f6bcd3336bf08d326546883854244768f80 and contender = cde5a0800624649cd6558f339ded2024146cfd71. cde5a0800624649cd6558f339ded2024146cfd71 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/799c02ca49794753848ba72c3ca738cb...e782b4e120994530bab1c8a2cd9bea4e/)
   [Failed :arrow_down:0.7% :arrow_up:0.04%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/dc466ac7d61b4be9b026f13acba67e59...5414362a17174b9186ea66b865378b50/)
   [Finished :arrow_down:0.0% :arrow_up:2.86%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7cd8f57cc3f14784af92d5ff4a59c196...6f6c4d9ea621465bb8579ebf1c350888/)
   [Finished :arrow_down:0.83% :arrow_up:0.04%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/f93dbaa6d111467baae708109466c54c...b8cf16926e7c4b0eb4bae5c4e77301ad/)
   Buildkite builds:
   [Finished] [`cde5a080` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/781)
   [Failed] [`cde5a080` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/778)
   [Finished] [`cde5a080` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/768)
   [Finished] [`cde5a080` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/784)
   [Finished] [`ed084f6b` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/780)
   [Failed] [`ed084f6b` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/777)
   [Finished] [`ed084f6b` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/767)
   [Finished] [`ed084f6b` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/783)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on PR #13112:
URL: https://github.com/apache/arrow/pull/13112#issuecomment-1128450803

   Change to draft. Wait for appveyor CI fixes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13112:
URL: https://github.com/apache/arrow/pull/13112#issuecomment-1127408099

   The added test fails on s390x:
   https://app.travis-ci.com/github/apache/arrow/jobs/569678317#L4649


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873587685


##########
cpp/src/arrow/util/io_util_test.cc:
##########
@@ -731,5 +732,31 @@ TEST(Memory, GetRSS) {
 #endif
 }
 
+// Some loose tests to check if the cpuinfo makes sense
+TEST(CpuInfo, Basic) {
+  const CpuInfo* ci = CpuInfo::GetInstance();
+
+  ASSERT_TRUE(ci->num_cores() >= 1 && ci->num_cores() <= 1000);
+
+  const auto l1 = ci->CacheSize(CpuInfo::CacheLevel::L1);
+  const auto l2 = ci->CacheSize(CpuInfo::CacheLevel::L2);
+  const auto l3 = ci->CacheSize(CpuInfo::CacheLevel::L3);
+  ASSERT_TRUE(l1 <= l2 && l2 <= l3);
+  ASSERT_TRUE(l1 >= 4 * 1024 && l1 <= 512 * 1024);
+  ASSERT_TRUE(l2 >= 32 * 1024 && l2 <= 8 * 1024 * 1024);
+  ASSERT_TRUE(l3 >= 256 * 1024 && l3 <= 128 * 1024 * 1024);
+
+  // Toggle hardware flags
+  CpuInfo* ci_rw = const_cast<CpuInfo*>(ci);
+  const int64_t original_hardware_flags = ci->hardware_flags();
+  ci_rw->EnableFeature(original_hardware_flags, false);
+  ASSERT_EQ(ci->hardware_flags(), 0);
+  ci_rw->EnableFeature(original_hardware_flags, true);
+  ASSERT_EQ(ci->hardware_flags(), original_hardware_flags);
+}
+
+// Dump detecetd cpu features in CI for manual check, not a real test
+TEST(CpuInfo, DISABLED_DebugMessage) { FAIL() << CpuInfo::GetInstance()->DebugMessage(); }

Review Comment:
   Removed this test and `CpuInfo::DebugMessage`. Not a useful api.



##########
cpp/src/arrow/util/cpu_info.cc:
##########
@@ -200,44 +93,59 @@ bool RetrieveCacheSize(int64_t* cache_sizes) {
           GetModuleHandle("kernel32"), "GetLogicalProcessorInformation");
 
   if (!func_pointer) {
-    return false;
+    return;

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cyb70289 commented on a diff in pull request #13112: ARROW-16478: [C++] Refine cpu info detection

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on code in PR #13112:
URL: https://github.com/apache/arrow/pull/13112#discussion_r873586429


##########
cpp/src/parquet/level_conversion.cc:
##########
@@ -137,7 +137,9 @@ void DefLevelsToBitmap(const int16_t* def_levels, int64_t num_def_levels,
   // is deleted in a follow-up release.
   if (level_info.rep_level > 0) {
 #if defined(ARROW_HAVE_RUNTIME_BMI2)
-    if (CpuInfo::GetInstance()->HasEfficientBmi2()) {
+    const CpuInfo* ci = CpuInfo::GetInstance();
+    // BMI2 (pext, pdep) is only efficient on Intel X86 processors.

Review Comment:
   Restored `HasEfficientBmi2`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org