You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cm...@apache.org on 2015/03/30 23:02:56 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/trunk 0967b1d99 -> d9ac5ee2c


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


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d9ac5ee2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d9ac5ee2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d9ac5ee2

Branch: refs/heads/trunk
Commit: d9ac5ee2c4dcd4a108ca892af501618caaea450c
Parents: 0967b1d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Mar 30 13:55:02 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Mar 30 13:55:02 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop-common/src/CMakeLists.txt            |  10 +
 .../src/org/apache/hadoop/util/bulk_crc32.c     | 561 +++----------------
 .../org/apache/hadoop/util/bulk_crc32_aarch64.c | 362 ++++++++++++
 .../src/org/apache/hadoop/util/bulk_crc32_x86.c | 345 ++++++++++++
 .../org/apache/hadoop/util/test_bulk_crc32.c    |  34 ++
 .../org/apache/hadoop/util/TestNativeCrc32.java |  16 +
 7 files changed, 840 insertions(+), 491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index b5d2303..13d74fd 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -471,6 +471,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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 942b19c..7d68fd7 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/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)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c
index c7efb8d..b3bb699 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c
+++ b/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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_aarch64.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_aarch64.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_aarch64.c
new file mode 100644
index 0000000..ab4690b
--- /dev/null
+++ b/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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_x86.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_x86.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_x86.c
new file mode 100644
index 0000000..290b8a6
--- /dev/null
+++ b/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;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c b/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c
index 5a8c9f2..ef3dbec 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c
+++ b/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;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9ac5ee2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java
index aecdc8f..ecc6c90 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java
@@ -97,6 +97,22 @@ public class TestNativeCrc32 {
   }
 
   @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();
     fillDataAndValidChecksums();