Переглянути джерело

HADOOP-11660. Add support for hardware crc of HDFS checksums on ARM aarch64 architecture (Edward Nevill via Colin P. McCabe)

(cherry picked from commit d9ac5ee2c4dcd4a108ca892af501618caaea450c)
Colin Patrick Mccabe 10 роки тому
батько
коміт
0048ec452b

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -36,6 +36,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug().
     (Walter Su via ozawa)
 
+    HADOOP-11660. Add support for hardware crc of HDFS checksums on ARM aarch64
+    architecture (Edward Nevill via Colin P. McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 10 - 0
hadoop-common-project/hadoop-common/src/CMakeLists.txt

@@ -163,6 +163,14 @@ else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
     ENDIF(REQUIRE_SNAPPY)
 endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
 
+IF (CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_x86.c")
+ELSEIF (CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_aarch64.c")
+ELSE()
+  MESSAGE("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
+ENDIF()
+
 # Find the no-suffix version of libcrypto.
 # See HADOOP-11216 for details.
 SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
@@ -228,6 +236,7 @@ CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
 add_executable(test_bulk_crc32
     ${D}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
     ${T}/util/test_bulk_crc32.c
 )
 
@@ -256,6 +265,7 @@ add_dual_library(hadoop
     ${D}/util/NativeCodeLoader.c
     ${D}/util/NativeCrc32.c
     ${D}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
 )
 if (NEED_LINK_DL)
    set(LIB_DL dl)

+ 70 - 491
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c

@@ -38,22 +38,23 @@
 #include "bulk_crc32.h"
 #include "gcc_optimizations.h"
 
-#if (!defined(__FreeBSD__) && !defined(WINDOWS))
-#define USE_PIPELINED
-#endif
-
 #define CRC_INITIAL_VAL 0xffffffff
 
-typedef uint32_t (*crc_update_func_t)(uint32_t, const uint8_t *, size_t);
 static uint32_t crc_val(uint32_t crc);
-static uint32_t crc32_zlib_sb8(uint32_t crc, const uint8_t *buf, size_t length);
-static uint32_t crc32c_sb8(uint32_t crc, const uint8_t *buf, size_t length);
 
-#ifdef USE_PIPELINED
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks);
-#endif
-static int cached_cpu_supports_crc32; // initialized by constructor below
-static uint32_t crc32c_hardware(uint32_t crc, const uint8_t* data, size_t length);
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+
+// The software versions of pipelined crc
+static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
+static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
+
+// Satically initialise the function pointers to the software versions
+// If a HW implementation is available they will subsequently be initialised in the dynamic
+// initialisers to point to the HW routines.
+crc_pipelined_func_t pipelined_crc32c_func = pipelined_crc32c_sb8;
+crc_pipelined_func_t pipelined_crc32_zlib_func = pipelined_crc32_zlib_sb8;
 
 static inline int store_or_verify(uint32_t *sums, uint32_t crc,
                                    int is_verify) {
@@ -72,94 +73,66 @@ int bulk_crc(const uint8_t *data, size_t data_len,
 
   int is_verify = error_info != NULL;
 
-#ifdef USE_PIPELINED
   uint32_t crc1, crc2, crc3;
   int n_blocks = data_len / bytes_per_checksum;
   int remainder = data_len % bytes_per_checksum;
-  int do_pipelined = 0;
-#endif
   uint32_t crc;
-  crc_update_func_t crc_update_func;
+  crc_pipelined_func_t crc_pipelined_func;
   switch (checksum_type) {
     case CRC32_ZLIB_POLYNOMIAL:
-      crc_update_func = crc32_zlib_sb8;
+      crc_pipelined_func = pipelined_crc32_zlib_func;
       break;
     case CRC32C_POLYNOMIAL:
-      if (likely(cached_cpu_supports_crc32)) {
-        crc_update_func = crc32c_hardware;
-#ifdef USE_PIPELINED
-        do_pipelined = 1;
-#endif
-      } else {
-        crc_update_func = crc32c_sb8;
-      }
+      crc_pipelined_func = pipelined_crc32c_func;
       break;
     default:
       return is_verify ? INVALID_CHECKSUM_TYPE : -EINVAL;
   }
 
-#ifdef USE_PIPELINED
-  if (do_pipelined) {
-    /* Process three blocks at a time */
-    while (likely(n_blocks >= 3)) {
-      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-      pipelined_crc32c(&crc1, &crc2, &crc3, data, bytes_per_checksum, 3);
+  /* Process three blocks at a time */
+  while (likely(n_blocks >= 3)) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, 3);
 
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
-        goto return_crc_error;
-      sums++;
-      data += bytes_per_checksum;
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
-        goto return_crc_error;
-      sums++;
-      data += bytes_per_checksum;
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc3))), is_verify)))
-        goto return_crc_error;
-      sums++;
-      data += bytes_per_checksum;
-      n_blocks -= 3;
-    }
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc3))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    n_blocks -= 3;
+  }
 
-    /* One or two blocks */
-    if (n_blocks) {
-      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-      pipelined_crc32c(&crc1, &crc2, &crc3, data, bytes_per_checksum, n_blocks);
+  /* One or two blocks */
+  if (n_blocks) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, n_blocks);
 
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+      goto return_crc_error;
+    data += bytes_per_checksum;
+    sums++;
+    if (n_blocks == 2) {
+      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
         goto return_crc_error;
-      data += bytes_per_checksum;
       sums++;
-      if (n_blocks == 2) {
-        if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
-          goto return_crc_error;
-        sums++;
-        data += bytes_per_checksum;
-      }
-    }
- 
-    /* For something smaller than a block */
-    if (remainder) {
-      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-      pipelined_crc32c(&crc1, &crc2, &crc3, data, remainder, 1);
-
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
-        goto return_crc_error;
+      data += bytes_per_checksum;
     }
-    return is_verify ? CHECKSUMS_VALID : 0;
   }
-#endif
 
-  while (likely(data_len > 0)) {
-    int len = likely(data_len >= bytes_per_checksum) ? bytes_per_checksum : data_len;
-    crc = CRC_INITIAL_VAL;
-    crc = crc_update_func(crc, data, len);
-    crc = ntohl(crc_val(crc));
-    if (unlikely(!store_or_verify(sums, crc, is_verify))) {
+  /* For something smaller than a block */
+  if (remainder) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, remainder, 1);
+
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
       goto return_crc_error;
-    }
-    data += len;
-    data_len -= len;
-    sums++;
   }
   return is_verify ? CHECKSUMS_VALID : 0;
 
@@ -175,7 +148,7 @@ return_crc_error:
 /**
  * Extract the final result of a CRC
  */
-uint32_t crc_val(uint32_t crc) {
+static uint32_t crc_val(uint32_t crc) {
   return ~crc;
 }
 
@@ -214,6 +187,16 @@ static uint32_t crc32c_sb8(uint32_t crc, const uint8_t *buf, size_t length) {
   return crc;    
 }
 
+static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
+  *crc1 = crc32c_sb8(*crc1, p_buf, block_size);
+  if (num_blocks >= 2)
+    *crc2 = crc32c_sb8(*crc2, p_buf+block_size, block_size);
+  if (num_blocks >= 3)
+    *crc3 = crc32c_sb8(*crc3, p_buf+2*block_size, block_size);
+}
+
 /**
  * Update a CRC using the "zlib" polynomial -- what Hadoop calls CHECKSUM_CRC32
  * using slicing-by-8
@@ -250,416 +233,12 @@ static uint32_t crc32_zlib_sb8(
   return crc;    
 }
 
-///////////////////////////////////////////////////////////////////////////
-// Begin code for SSE4.2 specific hardware support of CRC32C
-///////////////////////////////////////////////////////////////////////////
-
-#if (defined(__amd64__) || defined(__i386)) && defined(__GNUC__) && !defined(__FreeBSD__)
-#  define SSE42_FEATURE_BIT (1 << 20)
-#  define CPUID_FEATURES 1
-/**
- * Call the cpuid instruction to determine CPU feature flags.
- */
-static uint32_t cpuid(uint32_t eax_in) {
-  uint32_t eax, ebx, ecx, edx;
-#  if defined(__PIC__) && !defined(__LP64__)
-// 32-bit PIC code uses the ebx register for the base offset --
-// have to save and restore it on the stack
-  asm("pushl %%ebx\n\t"
-      "cpuid\n\t"
-      "movl %%ebx, %[ebx]\n\t"
-      "popl %%ebx" : "=a" (eax), [ebx] "=r"(ebx),  "=c"(ecx), "=d"(edx) : "a" (eax_in)
-      : "cc");
-#  else
-  asm("cpuid" : "=a" (eax), "=b"(ebx), "=c"(ecx), "=d"(edx) : "a"(eax_in)
-      : "cc");
-#  endif
-
-  return ecx;
-}
-
-/**
- * On library load, initiailize the cached value above for
- * whether the cpu supports SSE4.2's crc32 instruction.
- */
-void __attribute__ ((constructor)) init_cpu_support_flag(void) {
-  uint32_t ecx = cpuid(CPUID_FEATURES);
-  cached_cpu_supports_crc32 = ecx & SSE42_FEATURE_BIT;
-}
-
-
-//
-// Definitions of the SSE4.2 crc32 operations. Using these instead of
-// the GCC __builtin_* intrinsics allows this code to compile without
-// -msse4.2, since we do dynamic CPU detection at runtime.
-//
-
-#  ifdef __LP64__
-inline uint64_t _mm_crc32_u64(uint64_t crc, uint64_t value) {
-  asm("crc32q %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-#  endif
-
-inline uint32_t _mm_crc32_u32(uint32_t crc, uint32_t value) {
-  asm("crc32l %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-
-inline uint32_t _mm_crc32_u16(uint32_t crc, uint16_t value) {
-  asm("crc32w %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-
-inline uint32_t _mm_crc32_u8(uint32_t crc, uint8_t value) {
-  asm("crc32b %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
+static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                     const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
+  *crc1 = crc32_zlib_sb8(*crc1, p_buf, block_size);
+  if (num_blocks >= 2)
+    *crc2 = crc32_zlib_sb8(*crc2, p_buf+block_size, block_size);
+  if (num_blocks >= 3)
+    *crc3 = crc32_zlib_sb8(*crc3, p_buf+2*block_size, block_size);
 }
- 
-
-#  ifdef __LP64__
-/**
- * Hardware-accelerated CRC32C calculation using the 64-bit instructions.
- */
-static uint32_t crc32c_hardware(uint32_t crc, const uint8_t* p_buf, size_t length) {
-  // start directly at p_buf, even if it's an unaligned address. According
-  // to the original author of this code, doing a small run of single bytes
-  // to word-align the 64-bit instructions doesn't seem to help, but
-  // we haven't reconfirmed those benchmarks ourselves.
-  uint64_t crc64bit = crc;
-  size_t i;
-  for (i = 0; i < length / sizeof(uint64_t); i++) {
-    crc64bit = _mm_crc32_u64(crc64bit, *(uint64_t*) p_buf);
-    p_buf += sizeof(uint64_t);
-  }
-
-  // This ugly switch is slightly faster for short strings than the straightforward loop
-  uint32_t crc32bit = (uint32_t) crc64bit;
-  length &= sizeof(uint64_t) - 1;
-  switch (length) {
-    case 7:
-      crc32bit = _mm_crc32_u8(crc32bit, *p_buf++);
-    case 6:
-      crc32bit = _mm_crc32_u16(crc32bit, *(uint16_t*) p_buf);
-      p_buf += 2;
-    // case 5 is below: 4 + 1
-    case 4:
-      crc32bit = _mm_crc32_u32(crc32bit, *(uint32_t*) p_buf);
-      break;
-    case 3:
-      crc32bit = _mm_crc32_u8(crc32bit, *p_buf++);
-    case 2:
-      crc32bit = _mm_crc32_u16(crc32bit, *(uint16_t*) p_buf);
-      break;
-    case 5:
-      crc32bit = _mm_crc32_u32(crc32bit, *(uint32_t*) p_buf);
-      p_buf += 4;
-    case 1:
-      crc32bit = _mm_crc32_u8(crc32bit, *p_buf);
-      break;
-    case 0:
-      break;
-    default:
-      // This should never happen; enable in debug code
-      assert(0 && "ended up with 8 or more bytes at tail of calculation");
-  }
-
-  return crc32bit;
-}
-
-#ifdef USE_PIPELINED
-/**
- * Pipelined version of hardware-accelerated CRC32C calculation using
- * the 64 bit crc32q instruction. 
- * One crc32c instruction takes three cycles, but two more with no data
- * dependency can be in the pipeline to achieve something close to single 
- * instruction/cycle. Here we feed three blocks in RR.
- *
- *   crc1, crc2, crc3 : Store initial checksum for each block before
- *           calling. When it returns, updated checksums are stored.
- *   p_buf : The base address of the data buffer. The buffer should be
- *           at least as big as block_size * num_blocks.
- *   block_size : The size of each block in bytes.
- *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
- */
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  uint64_t c1 = *crc1;
-  uint64_t c2 = *crc2;
-  uint64_t c3 = *crc3;
-  uint64_t *data = (uint64_t*)p_buf;
-  int counter = block_size / sizeof(uint64_t);
-  int remainder = block_size % sizeof(uint64_t);
-  uint8_t *bdata;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down. Calling
-   * individual crc32 instructions three times from C also causes
-   * gcc to insert other instructions inbetween.
-   *
-   * Do not rearrange the following code unless you have verified
-   * the generated machine code is as efficient as before.
-   */
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%7), %0;\n\t"
-        "crc32q (%7,%6,1), %1;\n\t"
-        "crc32q (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      /* Take care of the remainder. They are only up to seven bytes,
-       * so performing byte-level crc32 won't take much time.
-       */
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%7), %0;\n\t"
-        "crc32b (%7,%6,1), %1;\n\t"
-        "crc32b (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%5), %0;\n\t"
-        "crc32q (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "0"(c1), "1"(c2), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%5), %0;\n\t"
-        "crc32b (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "0"(c1), "1"(c2), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 1:
-      /* single block */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "0"(c1), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "0"(c1), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 0:
-      return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-#endif /* USE_PIPELINED */
-
-# else  // 32-bit
-
-/**
- * Hardware-accelerated CRC32C calculation using the 32-bit instructions.
- */
-static uint32_t crc32c_hardware(uint32_t crc, const uint8_t* p_buf, size_t length) {
-  // start directly at p_buf, even if it's an unaligned address. According
-  // to the original author of this code, doing a small run of single bytes
-  // to word-align the 64-bit instructions doesn't seem to help, but
-  // we haven't reconfirmed those benchmarks ourselves.
-  size_t i;
-  for (i = 0; i < length / sizeof(uint32_t); i++) {
-    crc = _mm_crc32_u32(crc, *(uint32_t*) p_buf);
-    p_buf += sizeof(uint32_t);
-  }
-
-  // This ugly switch is slightly faster for short strings than the straightforward loop
-  length &= sizeof(uint32_t) - 1;
-  switch (length) {
-    case 3:
-      crc = _mm_crc32_u8(crc, *p_buf++);
-    case 2:
-      crc = _mm_crc32_u16(crc, *(uint16_t*) p_buf);
-      break;
-    case 1:
-      crc = _mm_crc32_u8(crc, *p_buf);
-      break;
-    case 0:
-      break;
-    default:
-      // This should never happen; enable in debug code
-      assert(0 && "ended up with 4 or more bytes at tail of calculation");
-  }
-
-  return crc;
-}
-
-#ifdef USE_PIPELINED
-/**
- * Pipelined version of hardware-accelerated CRC32C calculation using
- * the 32 bit crc32l instruction. 
- * One crc32c instruction takes three cycles, but two more with no data
- * dependency can be in the pipeline to achieve something close to single 
- * instruction/cycle. Here we feed three blocks in RR.
- *
- *   crc1, crc2, crc3 : Store initial checksum for each block before
- *                calling. When it returns, updated checksums are stored.
- *   data       : The base address of the data buffer. The buffer should be
- *                at least as big as block_size * num_blocks.
- *   block_size : The size of each block in bytes. 
- *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
- */
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  uint32_t c1 = *crc1;
-  uint32_t c2 = *crc2;
-  uint32_t c3 = *crc3;
-  int counter = block_size / sizeof(uint32_t);
-  int remainder = block_size % sizeof(uint32_t);
-  uint32_t *data = (uint32_t*)p_buf;
-  uint8_t *bdata;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down. Calling
-   * individual crc32 instructions three times from C also causes
-   * gcc to insert other instructions inbetween.
-   *
-   * Do not rearrange the following code unless you have verified
-   * the generated machine code is as efficient as before.
-   */
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%7), %0;\n\t"
-        "crc32l (%7,%6,1), %1;\n\t"
-        "crc32l (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      /* Take care of the remainder. They are only up to three bytes,
-       * so performing byte-level crc32 won't take much time.
-       */
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%7), %0;\n\t"
-        "crc32b (%7,%6,1), %1;\n\t"
-        "crc32b (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%5), %0;\n\t"
-        "crc32l (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%5), %0;\n\t"
-        "crc32b (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 1:
-      /* single block */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "r"(c1), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "r"(c1), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 0:
-       return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-
-#endif /* USE_PIPELINED */
-
-# endif // 64-bit vs 32-bit
-
-#else // end x86 architecture
-
-static uint32_t crc32c_hardware(uint32_t crc, const uint8_t* data, size_t length) {
-  // never called!
-  assert(0 && "hardware crc called on an unsupported platform");
-  return 0;
-}
-
-#endif

+ 362 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_aarch64.c

@@ -0,0 +1,362 @@
+/*
+ * 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 <assert.h>
+#include <stddef.h>    // for size_t
+
+#include  "bulk_crc32.h"
+#include "gcc_optimizations.h"
+
+/**
+ * Hardware-accelerated CRC32 calculation using the 64-bit instructions.
+ * 2 variants:-
+ *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
+ *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
+ */
+
+// gcc doesn't know how to vectorize a 128 bit load, so use the following to tell it
+#define LDP(x,y,p) asm("ldp %x[a], %x[b], [%x[c]], #16" : [a]"=r"(x),[b]"=r"(y),[c]"+r"(p))
+
+#define CRC32CX(crc,value) asm("crc32cx %w[c], %w[c], %x[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CW(crc,value) asm("crc32cw %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CH(crc,value) asm("crc32ch %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CB(crc,value) asm("crc32cb %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+
+#define CRC32ZX(crc,value) asm("crc32x %w[c], %w[c], %x[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZW(crc,value) asm("crc32w %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZH(crc,value) asm("crc32h %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZB(crc,value) asm("crc32b %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+
+/**
+ * Pipelined version of hardware-accelerated CRC32 calculation using
+ * the 64 bit crc32 instructions. 
+ * One crc32 instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ * 2 variants:-
+ *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
+ *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *           calling. When it returns, updated checksums are stored.
+ *   p_buf : The base address of the data buffer. The buffer should be
+ *           at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes.
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  const uint8_t *p_buf2 = p_buf1 + block_size;
+  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
+  uint64_t x1, y1, x2, y2, x3, y3;
+  long len = block_size;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down.
+   *
+   * Do verify that this code generates the expected assembler
+   * by disassembling test_bulk_crc32
+   */
+
+  asm(".cpu generic+crc");	// Allow crc instructions in asm
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        LDP(x3,y3,p_buf3);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c3, x3);
+        CRC32CX(c1, y1);
+        CRC32CX(c2, y2);
+        CRC32CX(c3, y3);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+        CRC32CW(c2, x2);
+        CRC32CW(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+        CRC32CH(c2, x2);
+        CRC32CH(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        x3 = *p_buf3;
+        CRC32CB(c1, x1);
+        CRC32CB(c2, x2);
+        CRC32CB(c3, x3);
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c1, y1);
+        CRC32CX(c2, y2);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+        CRC32CW(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+        CRC32CH(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        CRC32CB(c1, x1);
+        CRC32CB(c2, x2);
+      }
+      break;
+    case 1:
+      /* single block */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        CRC32CX(c1, x1);
+        CRC32CX(c1, y1);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        CRC32CB(c1, x1);
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+static void pipelined_crc32_zlib(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  const uint8_t *p_buf2 = p_buf1 + block_size;
+  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
+  uint64_t x1, y1, x2, y2, x3, y3;
+  long len = block_size;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down.
+   *
+   * Do verify that this code generates the expected assembler
+   * by disassembling test_bulk_crc32
+   */
+
+  asm(".cpu generic+crc");	// Allow crc instructions in asm
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        LDP(x3,y3,p_buf3);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c3, x3);
+        CRC32ZX(c1, y1);
+        CRC32ZX(c2, y2);
+        CRC32ZX(c3, y3);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+        CRC32ZW(c2, x2);
+        CRC32ZW(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+        CRC32ZH(c2, x2);
+        CRC32ZH(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        x3 = *p_buf3;
+        CRC32ZB(c1, x1);
+        CRC32ZB(c2, x2);
+        CRC32ZB(c3, x3);
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c1, y1);
+        CRC32ZX(c2, y2);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+        CRC32ZW(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+        CRC32ZH(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        CRC32ZB(c1, x1);
+        CRC32ZB(c2, x2);
+      }
+      break;
+    case 1:
+      /* single block */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c1, y1);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        CRC32ZB(c1, x1);
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+extern crc_pipelined_func_t pipelined_crc32c_func;
+extern crc_pipelined_func_t pipelined_crc32_zlib_func;
+
+#include <sys/auxv.h>
+#include <asm/hwcap.h>
+
+#ifndef HWCAP_CRC32
+#define HWCAP_CRC32 (1 << 7)
+#endif
+
+/**
+ * On library load, determine what sort of crc we are going to do
+ * and set crc function pointers appropriately.
+ */
+void __attribute__ ((constructor)) init_cpu_support_flag(void) {
+  unsigned long auxv = getauxval(AT_HWCAP);
+  if (auxv & HWCAP_CRC32) {
+    pipelined_crc32c_func = pipelined_crc32c;
+    pipelined_crc32_zlib_func = pipelined_crc32_zlib;
+  }
+}

+ 345 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_x86.c

@@ -0,0 +1,345 @@
+/*
+ * 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.
+ *
+ * Portions of this file are from http://www.evanjones.ca/crc32c.html under
+ * the BSD license:
+ *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
+ *   All rights reserved. Use of this source code is governed by a
+ *   BSD-style license that can be found in the LICENSE file.
+ */
+
+#include <assert.h>
+#include <stddef.h>    // for size_t
+
+#include  "bulk_crc32.h"
+#include "gcc_optimizations.h"
+#include "gcc_optimizations.h"
+
+///////////////////////////////////////////////////////////////////////////
+// Begin code for SSE4.2 specific hardware support of CRC32C
+///////////////////////////////////////////////////////////////////////////
+
+#  define SSE42_FEATURE_BIT (1 << 20)
+#  define CPUID_FEATURES 1
+/**
+ * Call the cpuid instruction to determine CPU feature flags.
+ */
+static uint32_t cpuid(uint32_t eax_in) {
+  uint32_t eax, ebx, ecx, edx;
+#  if defined(__PIC__) && !defined(__LP64__)
+// 32-bit PIC code uses the ebx register for the base offset --
+// have to save and restore it on the stack
+  asm("pushl %%ebx\n\t"
+      "cpuid\n\t"
+      "movl %%ebx, %[ebx]\n\t"
+      "popl %%ebx" : "=a" (eax), [ebx] "=r"(ebx),  "=c"(ecx), "=d"(edx) : "a" (eax_in)
+      : "cc");
+#  else
+  asm("cpuid" : "=a" (eax), "=b"(ebx), "=c"(ecx), "=d"(edx) : "a"(eax_in)
+      : "cc");
+#  endif
+
+  return ecx;
+}
+
+//
+// Definitions of the SSE4.2 crc32 operations. Using these instead of
+// the GCC __builtin_* intrinsics allows this code to compile without
+// -msse4.2, since we do dynamic CPU detection at runtime.
+//
+
+#  ifdef __LP64__
+inline uint64_t _mm_crc32_u64(uint64_t crc, uint64_t value) {
+  asm("crc32q %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+#  endif
+
+inline uint32_t _mm_crc32_u32(uint32_t crc, uint32_t value) {
+  asm("crc32l %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+inline uint32_t _mm_crc32_u16(uint32_t crc, uint16_t value) {
+  asm("crc32w %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+inline uint32_t _mm_crc32_u8(uint32_t crc, uint8_t value) {
+  asm("crc32b %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+#  ifdef __LP64__
+/**
+ * Pipelined version of hardware-accelerated CRC32C calculation using
+ * the 64 bit crc32q instruction. 
+ * One crc32c instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *           calling. When it returns, updated checksums are stored.
+ *   p_buf : The base address of the data buffer. The buffer should be
+ *           at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes.
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  uint64_t *data = (uint64_t*)p_buf;
+  int counter = block_size / sizeof(uint64_t);
+  int remainder = block_size % sizeof(uint64_t);
+  uint8_t *bdata;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down. Calling
+   * individual crc32 instructions three times from C also causes
+   * gcc to insert other instructions inbetween.
+   *
+   * Do not rearrange the following code unless you have verified
+   * the generated machine code is as efficient as before.
+   */
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%7), %0;\n\t"
+        "crc32q (%7,%6,1), %1;\n\t"
+        "crc32q (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      /* Take care of the remainder. They are only up to seven bytes,
+       * so performing byte-level crc32 won't take much time.
+       */
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%7), %0;\n\t"
+        "crc32b (%7,%6,1), %1;\n\t"
+        "crc32b (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%5), %0;\n\t"
+        "crc32q (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "0"(c1), "1"(c2), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%5), %0;\n\t"
+        "crc32b (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "0"(c1), "1"(c2), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 1:
+      /* single block */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "0"(c1), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "0"(c1), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+# else  // 32-bit
+
+/**
+ * Pipelined version of hardware-accelerated CRC32C calculation using
+ * the 32 bit crc32l instruction. 
+ * One crc32c instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *                calling. When it returns, updated checksums are stored.
+ *   data       : The base address of the data buffer. The buffer should be
+ *                at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes. 
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  uint32_t c1 = *crc1;
+  uint32_t c2 = *crc2;
+  uint32_t c3 = *crc3;
+  int counter = block_size / sizeof(uint32_t);
+  int remainder = block_size % sizeof(uint32_t);
+  uint32_t *data = (uint32_t*)p_buf;
+  uint8_t *bdata;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down. Calling
+   * individual crc32 instructions three times from C also causes
+   * gcc to insert other instructions inbetween.
+   *
+   * Do not rearrange the following code unless you have verified
+   * the generated machine code is as efficient as before.
+   */
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%7), %0;\n\t"
+        "crc32l (%7,%6,1), %1;\n\t"
+        "crc32l (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      /* Take care of the remainder. They are only up to three bytes,
+       * so performing byte-level crc32 won't take much time.
+       */
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%7), %0;\n\t"
+        "crc32b (%7,%6,1), %1;\n\t"
+        "crc32b (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%5), %0;\n\t"
+        "crc32l (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%5), %0;\n\t"
+        "crc32b (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 1:
+      /* single block */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "r"(c1), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "r"(c1), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 0:
+       return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+# endif // 64-bit vs 32-bit
+
+/**
+ * On library load, initiailize the cached function pointer
+ * if cpu supports SSE4.2's crc32 instruction.
+ */
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+extern crc_pipelined_func_t pipelined_crc32c_func;
+
+void __attribute__ ((constructor)) init_cpu_support_flag(void) {
+  uint32_t ecx = cpuid(CPUID_FEATURES);
+  if (ecx & SSE42_FEATURE_BIT) pipelined_crc32c_func = pipelined_crc32c;
+}

+ 34 - 0
hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c

@@ -23,6 +23,7 @@
 #include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
+#include <time.h>
 
 #define EXPECT_ZERO(x) \
     do { \
@@ -57,6 +58,36 @@ static int testBulkVerifyCrc(int dataLen, int crcType, int bytesPerChecksum)
   return 0;
 }
 
+static int timeBulkCrc(int dataLen, int crcType, int bytesPerChecksum, int iterations)
+{
+  int i;
+  uint8_t *data;
+  uint32_t *sums;
+  crc32_error_t errorData;
+  clock_t start, fini;
+
+  data = malloc(dataLen);
+  for (i = 0; i < dataLen; i++) {
+    data[i] = (i % 16) + 1;
+  }
+  sums = calloc(sizeof(uint32_t),
+                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
+
+  start = clock();
+  for (i = 0; i < iterations; i++) {
+    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+				   bytesPerChecksum, NULL));
+    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+			      bytesPerChecksum, &errorData));
+  }
+  fini = clock();
+  printf("CRC %d bytes @ %d bytes per checksum X %d iterations = %g\n",
+          dataLen, bytesPerChecksum, iterations, (double)(fini-start)/CLOCKS_PER_SEC);
+  free(data);
+  free(sums);
+  return 0;
+}
+
 int main(int argc, char **argv)
 {
   /* Test running bulk_calculate_crc with some different algorithms and
@@ -74,6 +105,9 @@ int main(int argc, char **argv)
   EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 4));
   EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 4));
 
+  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32C_POLYNOMIAL, 512, 1000000));
+  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32_ZLIB_POLYNOMIAL, 512, 1000000));
+
   fprintf(stderr, "%s: SUCCESS.\n", argv[0]);
   return EXIT_SUCCESS;
 }

+ 16 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java

@@ -96,6 +96,22 @@ public class TestNativeCrc32 {
       checksums, data, fileName, BASE_POSITION);
   }
 
+  @Test
+  public void testVerifyChunkedSumsSuccessOddSize() throws ChecksumException {
+    // Test checksum with an odd number of bytes. This is a corner case that
+    // is often broken in checksum calculation, because there is an loop which
+    // handles an even multiple or 4 or 8 bytes and then some additional code
+    // to finish the few odd bytes at the end. This code can often be broken
+    // but is never tested because we are always calling it with an even value
+    // such as 512.
+    bytesPerChecksum--;
+    allocateDirectByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.verifyChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, data, fileName, BASE_POSITION);
+    bytesPerChecksum++;
+  }
+
   @Test
   public void testVerifyChunkedSumsByteArraySuccess() throws ChecksumException {
     allocateArrayByteBuffers();