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 wh...@apache.org on 2014/10/07 19:24:38 UTC

[28/33] git commit: HDFS-7011. Implement basic utilities for libhdfs3 (cmccabe)

HDFS-7011. Implement basic utilities for libhdfs3 (cmccabe)


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

Branch: refs/heads/HDFS-6994
Commit: d873425a16e27807bfebffc73799a144cfe3b4dc
Parents: e0566dd
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Oct 1 14:09:42 2014 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Oct 7 11:23:45 2014 -0600

----------------------------------------------------------------------
 .../src/contrib/libhdfs3/CMake/Options.cmake    |   5 +
 .../src/contrib/libhdfs3/src/common/Atomic.h    |  50 ++
 .../src/contrib/libhdfs3/src/common/BigEndian.h |  56 ++
 .../contrib/libhdfs3/src/common/CFileWrapper.cc | 103 +++
 .../src/contrib/libhdfs3/src/common/Checksum.h  |  63 ++
 .../src/contrib/libhdfs3/src/common/DateTime.h  |  64 ++
 .../contrib/libhdfs3/src/common/Exception.cc    |  82 +++
 .../src/contrib/libhdfs3/src/common/Exception.h | 504 ++++++++++++++
 .../libhdfs3/src/common/ExceptionInternal.cc    | 188 ++++++
 .../libhdfs3/src/common/ExceptionInternal.h     | 268 ++++++++
 .../contrib/libhdfs3/src/common/FileWrapper.h   |  87 +++
 .../src/contrib/libhdfs3/src/common/Function.h  |  51 ++
 .../src/contrib/libhdfs3/src/common/HWCrc32c.cc | 159 +++++
 .../src/contrib/libhdfs3/src/common/HWCrc32c.h  |  77 +++
 .../src/contrib/libhdfs3/src/common/Hash.cc     |  90 +++
 .../src/contrib/libhdfs3/src/common/Hash.h      | 142 ++++
 .../src/contrib/libhdfs3/src/common/Logger.cc   | 109 +++
 .../src/contrib/libhdfs3/src/common/Logger.h    |  61 ++
 .../src/contrib/libhdfs3/src/common/LruMap.h    | 119 ++++
 .../libhdfs3/src/common/MappedFileWrapper.cc    | 139 ++++
 .../src/contrib/libhdfs3/src/common/SWCrc32c.cc |  96 +++
 .../src/contrib/libhdfs3/src/common/SWCrc32c.h  |  56 ++
 .../libhdfs3/src/common/SessionConfig.cc        | 170 +++++
 .../contrib/libhdfs3/src/common/SessionConfig.h | 324 +++++++++
 .../src/contrib/libhdfs3/src/common/SharedPtr.h |  32 +
 .../contrib/libhdfs3/src/common/StackPrinter.cc | 670 +++++++++++++++++++
 .../contrib/libhdfs3/src/common/StackPrinter.h  |  38 ++
 .../contrib/libhdfs3/src/common/StringUtil.h    |  65 ++
 .../src/contrib/libhdfs3/src/common/Thread.cc   |  47 ++
 .../src/contrib/libhdfs3/src/common/Thread.h    |  98 +++
 .../contrib/libhdfs3/src/common/UnorderedMap.h  |  32 +
 .../libhdfs3/src/common/WritableUtils.cc        | 185 +++++
 .../contrib/libhdfs3/src/common/WritableUtils.h |  66 ++
 .../contrib/libhdfs3/src/common/WriteBuffer.cc  |  71 ++
 .../contrib/libhdfs3/src/common/WriteBuffer.h   | 236 +++++++
 .../contrib/libhdfs3/src/common/XmlConfig.cc    | 395 +++++++++++
 .../src/contrib/libhdfs3/src/common/XmlConfig.h | 182 +++++
 37 files changed, 5180 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Options.cmake
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Options.cmake b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Options.cmake
index 322ecdb..eb74bd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Options.cmake
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Options.cmake
@@ -35,8 +35,13 @@ IF(ENABLE_DEBUG STREQUAL ON)
 ELSE(ENABLE_DEBUG STREQUAL ON)
     SET(CMAKE_BUILD_TYPE RelWithDebInfo CACHE 
         STRING "Choose the type of build, options are: None Debug Release RelWithDebInfo MinSizeRel." FORCE)
+    SET(CMAKE_CXX_FLAGS_DEBUG "-g -O2" CACHE STRING "compiler flags for RelWithDebInfo" FORCE)
+    SET(CMAKE_C_FLAGS_DEBUG "-g -O2" CACHE STRING "compiler flags for debug" FORCE)
 ENDIF(ENABLE_DEBUG STREQUAL ON)
 
+SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-strict-aliasing")
+SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-strict-aliasing")
+
 IF(ENABLE_COVERAGE STREQUAL ON)
     INCLUDE(CodeCoverage)
 ENDIF(ENABLE_COVERAGE STREQUAL ON)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atomic.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atomic.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atomic.h
new file mode 100644
index 0000000..d5fb518
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atomic.h
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_ATOMIC_H_
+#define _HDFS_LIBHDFS3_COMMON_ATOMIC_H_
+
+#include "platform.h"
+
+#ifdef NEED_BOOST
+
+#include <boost/atomic.hpp>
+
+namespace hdfs {
+namespace internal {
+
+using boost::atomic;
+
+}
+}
+
+#else
+
+#include <atomic>
+
+namespace hdfs {
+namespace internal {
+
+using std::atomic;
+
+}
+}
+#endif
+
+#endif /* _HDFS_LIBHDFS3_COMMON_ATOMIC_H_ */
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/BigEndian.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/BigEndian.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/BigEndian.h
new file mode 100644
index 0000000..ab508ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/BigEndian.h
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_BIGENDIAN_H_
+#define _HDFS_LIBHDFS3_COMMON_BIGENDIAN_H_
+
+#include <arpa/inet.h>
+#include <stdint.h>
+#include <string.h>
+
+namespace hdfs {
+namespace internal {
+
+static inline int16_t ReadBigEndian16FromArray(const char * buffer) {
+    int16_t retval;
+    retval = ntohs(*reinterpret_cast<const int16_t *>(buffer));
+    return retval;
+}
+
+static inline int32_t ReadBigEndian32FromArray(const char * buffer) {
+    int32_t retval;
+    retval = ntohl(*reinterpret_cast<const int32_t *>(buffer));
+    return retval;
+}
+
+static inline char * WriteBigEndian16ToArray(int16_t value, char * buffer) {
+    int16_t bigValue = htons(value);
+    memcpy(buffer, reinterpret_cast<const char *>(&bigValue), sizeof(int16_t));
+    return buffer + sizeof(int16_t);
+}
+
+static inline char * WriteBigEndian32ToArray(int32_t value, char * buffer) {
+    int32_t bigValue = htonl(value);
+    memcpy(buffer, reinterpret_cast<const char *>(&bigValue), sizeof(int32_t));
+    return buffer + sizeof(int32_t);
+}
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_BIGENDIAN_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/CFileWrapper.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/CFileWrapper.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/CFileWrapper.cc
new file mode 100644
index 0000000..15eeffa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/CFileWrapper.cc
@@ -0,0 +1,103 @@
+/**
+ * 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 "FileWrapper.h"
+
+#include <limits>
+#include <string>
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+namespace hdfs {
+namespace internal {
+
+CFileWrapper::CFileWrapper() :
+    file(NULL) {
+}
+
+CFileWrapper::~CFileWrapper() {
+    close();
+}
+
+bool CFileWrapper::open(int fd, bool delegate) {
+    assert(false && "not implemented");
+    abort();
+    return false;
+}
+
+bool CFileWrapper::open(const std::string &path) {
+    this->path = path;
+    file = fopen(path.c_str(), "rb");
+    return NULL != file;
+}
+
+void CFileWrapper::close() {
+    if (NULL != file) {
+        fclose(file);
+        file = NULL;
+    }
+}
+
+const char *CFileWrapper::read(std::vector<char> &buffer, int32_t size) {
+    buffer.resize(size);
+    copy(&buffer[0], size);
+    return &buffer[0];
+}
+
+void CFileWrapper::copy(char *buffer, int32_t size) {
+    int32_t todo = size, done;
+
+    while (todo > 0) {
+        done = fread(buffer + (size - todo), sizeof(char), todo, file);
+
+        if (done < 0) {
+            THROW(HdfsIOException, "Cannot read file \"%s\", %s.", path.c_str(),
+                  GetSystemErrorInfo(errno));
+        } else if (0 == done) {
+            THROW(HdfsIOException, "Cannot read file \"%s\", End of file.",
+                  path.c_str());
+        }
+
+        todo -= done;
+    }
+}
+
+void CFileWrapper::seek(int64_t offset) {
+    assert(offset > 0);
+    int64_t todo = offset, batch;
+    bool seek_set = true;
+
+    while (todo > 0) {
+        batch = todo < std::numeric_limits<long>::max() ?
+                todo : std::numeric_limits<long>::max();
+        off_t rc = fseek(file, static_cast<long>(batch),
+                         seek_set ? SEEK_SET : SEEK_CUR);
+        seek_set = false;
+
+        if (rc != 0) {
+            THROW(HdfsIOException, "Cannot lseek file: %s, %s", path.c_str(),
+                  GetSystemErrorInfo(errno));
+        }
+
+        todo -= batch;
+    }
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Checksum.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Checksum.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Checksum.h
new file mode 100644
index 0000000..d377e6d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Checksum.h
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_CHECKSUM_H_
+#define _HDFS_LIBHDFS3_COMMON_CHECKSUM_H_
+
+#include <stdint.h>
+
+#define CHECKSUM_TYPE_SIZE 1
+#define CHECKSUM_BYTES_PER_CHECKSUM_SIZE 4
+#define CHECKSUM_TYPE_CRC32C 2
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * An abstract base CRC class.
+ */
+class Checksum {
+public:
+    /**
+     * @return Returns the current checksum value.
+     */
+    virtual uint32_t getValue() = 0;
+
+    /**
+     * Resets the checksum to its initial value.
+     */
+    virtual void reset() = 0;
+
+    /**
+     * Updates the current checksum with the specified array of bytes.
+     * @param b The buffer of data.
+     * @param len The buffer length.
+     */
+    virtual void update(const void * b, int len) = 0;
+
+    /**
+     * Destroy the instance.
+     */
+    virtual ~Checksum() {
+    }
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_CHECKSUM_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/DateTime.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/DateTime.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/DateTime.h
new file mode 100644
index 0000000..d848552
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/DateTime.h
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_DATETIME_H_
+#define _HDFS_LIBHDFS3_COMMON_DATETIME_H_
+
+#include "platform.h"
+
+#include <ctime>
+#include <cassert>
+
+#ifdef NEED_BOOST
+
+#include <boost/chrono.hpp>
+
+namespace hdfs {
+namespace internal {
+
+using namespace boost::chrono;
+
+}
+}
+
+#else
+
+#include <chrono>
+
+namespace hdfs {
+namespace internal {
+
+using namespace std::chrono;
+
+}
+}
+#endif
+
+namespace hdfs {
+namespace internal {
+
+template<typename TimeStamp>
+static int64_t ToMilliSeconds(TimeStamp const & s, TimeStamp const & e) {
+    assert(e >= s);
+    return duration_cast<milliseconds>(e - s).count();
+}
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_DATETIME_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.cc
new file mode 100644
index 0000000..c693ca1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.cc
@@ -0,0 +1,82 @@
+/**
+ * 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 "Exception.h"
+
+#include <sstream>
+
+namespace hdfs {
+
+const char *HdfsIOException::ReflexName = "java.io.IOException";
+
+const char *AlreadyBeingCreatedException::ReflexName =
+    "org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException";
+
+const char *AccessControlException::ReflexName =
+    "org.apache.hadoop.security.AccessControlException";
+
+const char *FileAlreadyExistsException::ReflexName =
+    "org.apache.hadoop.fs.FileAlreadyExistsException";
+
+const char *DSQuotaExceededException::ReflexName =
+    "org.apache.hadoop.hdfs.protocol.DSQuotaExceededException";
+
+const char *NSQuotaExceededException::ReflexName =
+    "org.apache.hadoop.hdfs.protocol.NSQuotaExceededException";
+
+const char *ParentNotDirectoryException::ReflexName =
+    "org.apache.hadoop.fs.ParentNotDirectoryException";
+
+const char *SafeModeException::ReflexName =
+    "org.apache.hadoop.hdfs.server.namenode.SafeModeException";
+
+const char *NotReplicatedYetException::ReflexName =
+    "org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException";
+
+const char *FileNotFoundException::ReflexName = "java.io.FileNotFoundException";
+
+const char *UnresolvedLinkException::ReflexName =
+    "org.apache.hadoop.fs.UnresolvedLinkException";
+
+const char *UnsupportedOperationException::ReflexName =
+    "java.lang.UnsupportedOperationException";
+
+const char *ReplicaNotFoundException::ReflexName =
+    "org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException";
+
+const char *NameNodeStandbyException::ReflexName =
+    "org.apache.hadoop.ipc.StandbyException";
+
+const char *HdfsInvalidBlockToken::ReflexName =
+    "org.apache.hadoop.security.token.SecretManager$InvalidToken";
+
+const char *SaslException::ReflexName = "javax.security.sasl.SaslException";
+
+const char *RpcNoSuchMethodException::ReflexName = "org.apache.hadoop.ipc.RpcNoSuchMethodException";
+
+const char *InvalidParameter::ReflexName = "java.lang.IllegalArgumentException";
+
+HdfsException::HdfsException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+    std::runtime_error(arg) {
+    std::ostringstream ss;
+    ss << file << ": " << line << ": " << arg << std::endl << stack;
+    detail = ss.str();
+}
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.h
new file mode 100644
index 0000000..71af81f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Exception.h
@@ -0,0 +1,504 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_EXCEPTION_H_
+#define _HDFS_LIBHDFS3_COMMON_EXCEPTION_H_
+
+#include <stdexcept>
+#include <string>
+
+namespace hdfs {
+
+class HdfsException: public std::runtime_error {
+public:
+    HdfsException(const std::string &arg, const char *file, int line,
+                  const char *stack);
+
+    ~HdfsException() throw () {
+    }
+
+    virtual const char *msg() const {
+        return detail.c_str();
+    }
+
+protected:
+    std::string detail;
+};
+
+class HdfsIOException: public HdfsException {
+public:
+    HdfsIOException(const std::string &arg, const char *file, int line,
+                    const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsIOException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class HdfsNetworkException: public HdfsIOException {
+public:
+    HdfsNetworkException(const std::string &arg, const char *file, int line,
+                         const char *stack) :
+        HdfsIOException(arg, file, line, stack) {
+    }
+
+    ~HdfsNetworkException() throw () {
+    }
+};
+
+class HdfsNetworkConnectException: public HdfsNetworkException {
+public:
+    HdfsNetworkConnectException(const std::string &arg, const char *file, int line,
+                                const char *stack) :
+        HdfsNetworkException(arg, file, line, stack) {
+    }
+
+    ~HdfsNetworkConnectException() throw () {
+    }
+};
+
+class AccessControlException: public HdfsException {
+public:
+    AccessControlException(const std::string &arg, const char *file, int line,
+                           const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~AccessControlException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class AlreadyBeingCreatedException: public HdfsException {
+public:
+    AlreadyBeingCreatedException(const std::string &arg, const char *file,
+                                 int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~AlreadyBeingCreatedException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class ChecksumException: public HdfsException {
+public:
+    ChecksumException(const std::string &arg, const char *file, int line,
+                      const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~ChecksumException() throw () {
+    }
+};
+
+class DSQuotaExceededException: public HdfsException {
+public:
+    DSQuotaExceededException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~DSQuotaExceededException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class FileAlreadyExistsException: public HdfsException {
+public:
+    FileAlreadyExistsException(const std::string &arg, const char *file,
+                               int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~FileAlreadyExistsException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class FileNotFoundException: public HdfsException {
+public:
+    FileNotFoundException(const std::string &arg, const char *file, int line,
+                          const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~FileNotFoundException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class HdfsBadBoolFoumat: public HdfsException {
+public:
+    HdfsBadBoolFoumat(const std::string &arg, const char *file, int line,
+                      const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsBadBoolFoumat() throw () {
+    }
+};
+
+class HdfsBadConfigFoumat: public HdfsException {
+public:
+    HdfsBadConfigFoumat(const std::string &arg, const char *file, int line,
+                        const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsBadConfigFoumat() throw () {
+    }
+};
+
+class HdfsBadNumFoumat: public HdfsException {
+public:
+    HdfsBadNumFoumat(const std::string &arg, const char *file, int line,
+                     const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsBadNumFoumat() throw () {
+    }
+};
+
+class HdfsCanceled: public HdfsException {
+public:
+    HdfsCanceled(const std::string &arg, const char *file, int line,
+                 const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsCanceled() throw () {
+    }
+};
+
+class HdfsFileSystemClosed: public HdfsException {
+public:
+    HdfsFileSystemClosed(const std::string &arg, const char *file, int line,
+                         const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsFileSystemClosed() throw () {
+    }
+};
+
+class HdfsConfigInvalid: public HdfsException {
+public:
+    HdfsConfigInvalid(const std::string &arg, const char *file, int line,
+                      const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsConfigInvalid() throw () {
+    }
+};
+
+class HdfsConfigNotFound: public HdfsException {
+public:
+    HdfsConfigNotFound(const std::string &arg, const char *file, int line,
+                       const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsConfigNotFound() throw () {
+    }
+};
+
+class HdfsEndOfStream: public HdfsIOException {
+public:
+    HdfsEndOfStream(const std::string &arg, const char *file, int line,
+                    const char *stack) :
+        HdfsIOException(arg, file, line, stack) {
+    }
+
+    ~HdfsEndOfStream() throw () {
+    }
+};
+
+class HdfsInvalidBlockToken: public HdfsException {
+public:
+    HdfsInvalidBlockToken(const std::string &arg, const char *file, int line,
+                          const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsInvalidBlockToken() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+/**
+ * This will wrap HdfsNetworkConnectionException and HdfsTimeoutException.
+ * This exception will be caught and attempt will be performed to recover in HA case.
+ */
+class HdfsFailoverException: public HdfsException {
+public:
+    HdfsFailoverException(const std::string &arg, const char *file, int line,
+                          const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsFailoverException() throw () {
+    }
+};
+
+/**
+ * Fatal error during the rpc call. It may wrap other exceptions.
+ */
+class HdfsRpcException: public HdfsIOException {
+public:
+    HdfsRpcException(const std::string &arg, const char *file, int line,
+                     const char *stack) :
+        HdfsIOException(arg, file, line, stack) {
+    }
+
+    ~HdfsRpcException() throw () {
+    }
+};
+
+/**
+ * Server throw an error during the rpc call.
+ * It should be used internally and parsed for details.
+ */
+class HdfsRpcServerException: public HdfsIOException {
+public:
+    HdfsRpcServerException(const std::string &arg, const char *file, int line,
+                           const char *stack) :
+        HdfsIOException(arg, file, line, stack) {
+    }
+
+    ~HdfsRpcServerException() throw () {
+    }
+
+    const std::string &getErrClass() const {
+        return errClass;
+    }
+
+    void setErrClass(const std::string &errClass) {
+        this->errClass = errClass;
+    }
+
+    const std::string &getErrMsg() const {
+        return errMsg;
+    }
+
+    void setErrMsg(const std::string &errMsg) {
+        this->errMsg = errMsg;
+    }
+
+private:
+    std::string errClass;
+    std::string errMsg;
+};
+
+class HdfsTimeoutException: public HdfsException {
+public:
+    HdfsTimeoutException(const std::string &arg, const char *file, int line,
+                         const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~HdfsTimeoutException() throw () {
+    }
+};
+
+class InvalidParameter: public HdfsException {
+public:
+    InvalidParameter(const std::string &arg, const char *file, int line,
+                     const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~InvalidParameter() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class InvalidPath: public HdfsException {
+public:
+    InvalidPath(const std::string &arg, const char *file, int line,
+                const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~InvalidPath() throw () {
+    }
+};
+
+class NotReplicatedYetException: public HdfsException {
+public:
+    NotReplicatedYetException(const std::string &arg, const char *file,
+                              int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~NotReplicatedYetException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class NSQuotaExceededException: public HdfsException {
+public:
+    NSQuotaExceededException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~NSQuotaExceededException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class ParentNotDirectoryException: public HdfsException {
+public:
+    ParentNotDirectoryException(const std::string &arg, const char *file,
+                                int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~ParentNotDirectoryException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class ReplicaNotFoundException: public HdfsException {
+public:
+    ReplicaNotFoundException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~ReplicaNotFoundException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class SafeModeException: public HdfsException {
+public:
+    SafeModeException(const std::string &arg, const char *file, int line,
+                      const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~SafeModeException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class UnresolvedLinkException: public HdfsException {
+public:
+    UnresolvedLinkException(const std::string &arg, const char *file,
+                            int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~UnresolvedLinkException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class UnsupportedOperationException: public HdfsException {
+public:
+    UnsupportedOperationException(const std::string &arg, const char *file,
+                                  int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~UnsupportedOperationException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class SaslException: public HdfsException {
+public:
+    SaslException(const std::string &arg, const char *file, int line,
+                  const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~SaslException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class NameNodeStandbyException: public HdfsException {
+public:
+    NameNodeStandbyException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~NameNodeStandbyException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+class RpcNoSuchMethodException: public HdfsException {
+public:
+    RpcNoSuchMethodException(const std::string &arg, const char *file,
+                             int line, const char *stack) :
+        HdfsException(arg, file, line, stack) {
+    }
+
+    ~RpcNoSuchMethodException() throw () {
+    }
+
+public:
+    static const char *ReflexName;
+};
+
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_EXCEPTION_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.cc
new file mode 100644
index 0000000..3279feb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.cc
@@ -0,0 +1,188 @@
+/**
+ * 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 "platform.h"
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "Thread.h"
+
+#include <cassert>
+#include <cstring>
+#include <sstream>
+
+namespace hdfs {
+
+function<bool(void)> ChecnOperationCanceledCallback;
+
+namespace internal {
+
+bool CheckOperationCanceled() {
+    if (ChecnOperationCanceledCallback && ChecnOperationCanceledCallback()) {
+        THROW(HdfsCanceled, "Operation has been canceled by the user.");
+    }
+
+    return false;
+}
+
+const char *GetSystemErrorInfo(int eno) {
+    static THREAD_LOCAL char message[64];
+    char buffer[64], *pbuffer;
+    pbuffer = buffer;
+#ifdef STRERROR_R_RETURN_INT
+    strerror_r(eno, buffer, sizeof(buffer));
+#else
+    pbuffer = strerror_r(eno, buffer, sizeof(buffer));
+#endif
+    snprintf(message, sizeof(message), "(errno: %d) %s", eno, pbuffer);
+    return message;
+}
+
+static THREAD_LOCAL std::string *MessageBuffer = NULL;
+
+static void InitMessageBuffer() {
+    if (!MessageBuffer) {
+      MessageBuffer = new std::string;
+    }
+}
+
+static void GetExceptionDetailInternal(const hdfs::HdfsException &e,
+                                       std::stringstream &ss, bool topLevel);
+
+static void GetExceptionDetailInternal(const std::exception &e,
+                                       std::stringstream &ss, bool topLevel) {
+    try {
+        if (!topLevel) {
+            ss << "Caused by\n";
+        }
+
+        ss << e.what();
+    } catch (const std::bad_alloc &e) {
+        return;
+    }
+
+    try {
+        hdfs::rethrow_if_nested(e);
+    } catch (const hdfs::HdfsException &nested) {
+        GetExceptionDetailInternal(nested, ss, false);
+    } catch (const std::exception &nested) {
+        GetExceptionDetailInternal(nested, ss, false);
+    }
+}
+
+static void GetExceptionDetailInternal(const hdfs::HdfsException &e,
+                                       std::stringstream &ss, bool topLevel) {
+    try {
+        if (!topLevel) {
+            ss << "Caused by\n";
+        }
+
+        ss << e.msg();
+    } catch (const std::bad_alloc &e) {
+        return;
+    }
+
+    try {
+        hdfs::rethrow_if_nested(e);
+    } catch (const hdfs::HdfsException &nested) {
+        GetExceptionDetailInternal(nested, ss, false);
+    } catch (const std::exception &nested) {
+        GetExceptionDetailInternal(nested, ss, false);
+    }
+}
+
+const char *GetExceptionDetail(const hdfs::HdfsException &e) {
+    std::stringstream ss;
+    GetExceptionDetailInternal(e, ss, true);
+
+    try {
+        InitMessageBuffer();
+        *MessageBuffer = ss.str();
+    } catch (const std::bad_alloc &e) {
+        return "Out of memory";
+    }
+
+    return MessageBuffer->c_str();
+}
+
+const char *GetExceptionDetail(const exception_ptr e) {
+    std::stringstream ss;
+
+    try {
+        InitMessageBuffer();
+        hdfs::rethrow_exception(e);
+    } catch (const hdfs::HdfsException &nested) {
+        GetExceptionDetailInternal(nested, ss, true);
+    } catch (const std::exception &nested) {
+        GetExceptionDetailInternal(nested, ss, true);
+    }
+
+    try {
+        *MessageBuffer = ss.str();
+    } catch (const std::bad_alloc &e) {
+        return "Out of memory";
+    }
+
+    return MessageBuffer->c_str();
+}
+
+static void GetExceptionMessage(const std::exception &e,
+                                std::stringstream &ss, int recursive) {
+    try {
+        for (int i = 0; i < recursive; ++i) {
+            ss << '\t';
+        }
+
+        if (recursive > 0) {
+            ss << "Caused by: ";
+        }
+
+        ss << e.what();
+    } catch (const std::bad_alloc &e) {
+        return;
+    }
+
+    try {
+        hdfs::rethrow_if_nested(e);
+    } catch (const std::exception &nested) {
+        GetExceptionMessage(nested, ss, recursive + 1);
+    }
+}
+
+const char *GetExceptionMessage(const exception_ptr e, std::string &buffer) {
+    std::stringstream ss;
+
+    try {
+        hdfs::rethrow_exception(e);
+    } catch (const std::bad_alloc &e) {
+        return "Out of memory";
+    } catch (const std::exception &e) {
+        GetExceptionMessage(e, ss, 0);
+    }
+
+    try {
+        buffer = ss.str();
+    } catch (const std::bad_alloc &e) {
+        return "Out of memory";
+    }
+
+    return buffer.c_str();
+}
+
+}
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.h
new file mode 100644
index 0000000..1cd6a02
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ExceptionInternal.h
@@ -0,0 +1,268 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_EXCEPTION_EXCEPTIONINTERNAL_H_
+#define _HDFS_LIBHDFS3_EXCEPTION_EXCEPTIONINTERNAL_H_
+
+#include "platform.h"
+
+#include <cassert>
+#include <cstdarg>
+#include <cstdio>
+#include <cstring>
+#include <unistd.h>
+#include <string>
+#include <sstream>
+
+#include "Function.h"
+#include "StackPrinter.h"
+
+#define STACK_DEPTH 64
+
+#define PATH_SEPRATOR '/'
+inline static const char *SkipPathPrefix(const char *path) {
+    int i, len = strlen(path);
+
+    for (i = len - 1; i > 0; --i) {
+        if (path[i] == PATH_SEPRATOR) {
+            break;
+        }
+    }
+
+    assert(i > 0 && i < len);
+    return path + i + 1;
+}
+
+#ifdef NEED_BOOST
+#include <boost/exception/all.hpp>
+
+namespace hdfs {
+using boost::exception_ptr;
+using boost::rethrow_exception;
+using boost::current_exception;
+
+class nested_exception : virtual public boost::exception {
+public:
+    nested_exception() : p(boost::current_exception()) {
+    }
+
+    nested_exception(const nested_exception &other) : p(other.p) {
+    }
+
+    nested_exception &operator = (const nested_exception &other) {
+        this->p = other.p;
+        return *this;
+    }
+
+    virtual ~nested_exception() throw() {}
+
+    void rethrow_nested() const {
+        boost::rethrow_exception(p);
+    }
+
+    boost::exception_ptr nested_ptr() const {
+        return p;
+    }
+protected:
+    boost::exception_ptr p;
+};
+
+template<typename BaseType>
+struct ExceptionWrapper : public BaseType, public nested_exception {
+    explicit ExceptionWrapper(BaseType const &e) : BaseType(static_cast < BaseType const &>(e)) {}
+    ~ExceptionWrapper() throw() {}
+};
+
+template<typename T>
+ATTRIBUTE_NORETURN
+static inline void throw_with_nested(T const &e) {
+    if (dynamic_cast<const nested_exception *>(&e)) {
+        std::terminate();
+    }
+
+    boost::throw_exception(ExceptionWrapper<T>(static_cast < T const &>(e)));
+}
+
+template<typename T>
+static inline void rethrow_if_nested(T const &e) {
+    const nested_exception *nested = dynamic_cast<const nested_exception *>(&e);
+
+    if (nested) {
+        nested->rethrow_nested();
+    }
+}
+
+template<typename T>
+static inline void rethrow_if_nested(const nested_exception &e) {
+    e.rethrow_nested();
+}
+
+namespace internal {
+
+
+template<typename THROWABLE>
+ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE
+void ThrowException(bool nested, const char *f, int l,
+                    const char *exceptionName, const char *fmt, ...) __attribute__((format(printf, 5, 6))) ;
+
+template<typename THROWABLE>
+ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE
+void ThrowException(bool nested, const char *f, int l,
+                    const char *exceptionName, const char *fmt, ...) {
+    va_list ap;
+    va_start(ap, fmt);
+    std::string buffer;
+    buffer = exceptionName;
+    buffer.append(": ");
+    int size = vsnprintf(NULL, 0, fmt, ap);
+    va_end(ap);
+    int offset = buffer.size();
+    buffer.resize(offset + size + 1);
+    va_start(ap, fmt);
+    vsnprintf(&buffer[offset], size + 1, fmt, ap);
+    va_end(ap);
+
+    if (!nested) {
+        boost::throw_exception(
+            THROWABLE(buffer.c_str(), SkipPathPrefix(f), l,
+                      hdfs::internal::PrintStack(1, STACK_DEPTH).c_str()));
+    } else {
+        hdfs::throw_with_nested(
+            THROWABLE(buffer.c_str(), SkipPathPrefix(f), l,
+                      hdfs::internal::PrintStack(1, STACK_DEPTH).c_str()));
+    }
+
+    throw std::logic_error("should not reach here.");
+}
+}
+
+}
+
+#else
+
+#include <exception>
+#include <stdexcept>
+
+namespace hdfs {
+
+using std::rethrow_exception;
+using std::current_exception;
+using std::make_exception_ptr;
+using std::throw_with_nested;
+using std::rethrow_if_nested;
+using std::exception_ptr;
+
+namespace internal {
+
+template<typename THROWABLE>
+ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE
+void ThrowException(bool nested, const char *f, int l,
+                    const char *exceptionName, const char *fmt, ...) __attribute__((format(printf, 5, 6)));
+
+template<typename THROWABLE>
+ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE
+void ThrowException(bool nested, const char *f, int l,
+                    const char *exceptionName, const char *fmt, ...) {
+    va_list ap;
+    va_start(ap, fmt);
+    std::string buffer;
+    buffer = exceptionName;
+    buffer.append(": ");
+    int size = vsnprintf(NULL, 0, fmt, ap);
+    va_end(ap);
+    int offset = buffer.size();
+    buffer.resize(offset + size + 1);
+    va_start(ap, fmt);
+    vsnprintf(&buffer[offset], size + 1, fmt, ap);
+    va_end(ap);
+
+    if (!nested) {
+        throw THROWABLE(buffer.c_str(), SkipPathPrefix(f), l,
+                        hdfs::internal::PrintStack(1, STACK_DEPTH).c_str());
+    } else {
+        std::throw_with_nested(
+            THROWABLE(buffer.c_str(), SkipPathPrefix(f), l,
+                      hdfs::internal::PrintStack(1, STACK_DEPTH).c_str()));
+    }
+
+    throw std::logic_error("should not reach here.");
+}
+
+}
+}
+
+#endif
+
+namespace hdfs {
+
+/**
+ * A user defined callback function used to check if a slow operation has been canceled by the user.
+ * If this function return true, HdfsCanceled will be thrown.
+ */
+extern function<bool(void)> ChecnOperationCanceledCallback;
+
+class HdfsException;
+
+}
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * Check if a slow operation has been canceled by the user.
+ * @throw return false if operation is not canceled, else throw HdfsCanceled.
+ * @throw HdfsCanceled
+ */
+bool CheckOperationCanceled();
+
+/**
+ * Get a exception's detail message.
+ * If the exception contains a nested exception, recursively get all the nested exception's detail message.
+ * @param e The exception which detail message to be return.
+ * @return The exception's detail message.
+ */
+const char *GetExceptionDetail(const hdfs::HdfsException &e);
+
+/**
+ * Get a exception's detail message.
+ * If the exception contains a nested exception, recursively get all the nested exception's detail message.
+ * @param e The exception which detail message to be return.
+ * @return The exception's detail message.
+ */
+const char *GetExceptionDetail(const exception_ptr e);
+
+const char *GetExceptionMessage(const exception_ptr e, std::string &buffer);
+
+/**
+ * Get a error information by the given system error number.
+ * @param eno System error number.
+ * @return The error information.
+ * @throw nothrow
+ */
+const char *GetSystemErrorInfo(int eno);
+
+}
+}
+
+#define THROW(throwable, fmt, ...) \
+    hdfs::internal::ThrowException<throwable>(false, __FILE__, __LINE__, #throwable, fmt, ##__VA_ARGS__);
+
+#define NESTED_THROW(throwable, fmt, ...) \
+    hdfs::internal::ThrowException<throwable>(true, __FILE__, __LINE__, #throwable, fmt, ##__VA_ARGS__);
+
+#endif /* _HDFS_LIBHDFS3_EXCEPTION_EXCEPTIONINTERNAL_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/FileWrapper.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/FileWrapper.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/FileWrapper.h
new file mode 100644
index 0000000..98c301f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/FileWrapper.h
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_FILEWRAPPER_H_
+#define _HDFS_LIBHDFS3_COMMON_FILEWRAPPER_H_
+
+#include <cassert>
+#include <cstdio>
+#include <stdint.h>
+#include <string>
+#include <string>
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+class FileWrapper {
+public:
+    virtual ~FileWrapper() {
+    }
+
+    virtual bool open(int fd, bool delegate) = 0;
+    virtual bool open(const std::string &path) = 0;
+    virtual void close() = 0;
+    virtual const char *read(std::vector<char> &buffer, int32_t size) = 0;
+    virtual void copy(char *buffer, int32_t size) = 0;
+    virtual void seek(int64_t position) = 0;
+};
+
+class CFileWrapper: public FileWrapper {
+public:
+    CFileWrapper();
+    ~CFileWrapper();
+    bool open(int fd, bool delegate);
+    bool open(const std::string &path);
+    void close();
+    const char *read(std::vector<char> &buffer, int32_t size);
+    void copy(char *buffer, int32_t size);
+    void seek(int64_t offset);
+
+private:
+    FILE *file;
+    std::string path;
+};
+
+class MappedFileWrapper: public FileWrapper {
+public:
+    MappedFileWrapper();
+    ~MappedFileWrapper();
+    bool open(int fd, bool delegate);
+    bool open(const std::string &path);
+    void close();
+    const char *read(std::vector<char> &buffer, int32_t size);
+    void copy(char *buffer, int32_t size);
+    void seek(int64_t offset);
+
+private:
+    bool openInternal(int fd, bool delegate, size_t size);
+
+private:
+    bool delegate;
+    const char *begin;
+    const char *position;
+    int fd;
+    int64_t size;
+    std::string path;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_FILEWRAPPER_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Function.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Function.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Function.h
new file mode 100644
index 0000000..124b64c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Function.h
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_FUNCTION_H_
+#define _HDFS_LIBHDFS3_COMMON_FUNCTION_H_
+
+#include "platform.h"
+
+#ifdef NEED_BOOST
+#include <boost/function.hpp>
+#include <boost/bind.hpp>
+
+namespace hdfs {
+
+using boost::function;
+using boost::bind;
+using boost::reference_wrapper;
+
+}
+
+#else
+
+#include <functional>
+
+namespace hdfs {
+
+using std::function;
+using std::bind;
+using std::reference_wrapper;
+using namespace std::placeholders;
+
+}
+
+#endif
+
+#endif /* _HDFS_LIBHDFS3_COMMON_FUNCTION_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.cc
new file mode 100644
index 0000000..a4542fa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.cc
@@ -0,0 +1,159 @@
+/**
+ * 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 <cassert>
+#include <cstdlib>
+
+#include "HWCrc32c.h"
+
+#if ((defined(__X86__) || defined(__i386__) || defined(i386) || \
+      defined(_M_IX86) || defined(__386__) || defined(__x86_64__) || \
+      defined(_M_X64)))
+#include <cpuid.h>
+#endif
+
+#if ((defined(__X86__) || defined(__i386__) || defined(i386) || \
+      defined(_M_IX86) || defined(__386__) || defined(__x86_64__) || \
+      defined(_M_X64)))
+#if !defined(__SSE4_2__)
+
+namespace hdfs {
+namespace internal {
+
+#if defined(__LP64__)
+static 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
+
+static 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;
+}
+
+static inline uint32_t _mm_crc32_u32(uint32_t crc, uint64_t value) {
+    asm("crc32l %[value], %[crc]\n" : [crc] "+r"(crc) : [value] "rm"(value));
+    return crc;
+}
+
+static 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;
+}
+
+}
+}
+
+#else
+
+#include <nmmintrin.h>
+
+#endif
+
+namespace hdfs {
+namespace internal {
+
+bool HWCrc32c::available() {
+#if ((defined(__X86__) || defined(__i386__) || defined(i386) || defined(_M_IX86) || defined(__386__) || defined(__x86_64__) || defined(_M_X64)))
+    uint32_t eax, ebx, ecx = 0, edx;
+    /*
+     * get the CPU features (level 1). ecx will have the SSE4.2 bit.
+     * This gcc routine automatically handles saving ebx in the case where we are -fpic or -fPIC
+     */
+    __get_cpuid(1, &eax, &ebx, &ecx, &edx);
+    return (ecx & (1 << 20)) != 0;
+#else
+    return false;
+#endif
+}
+
+void HWCrc32c::update(const void * b, int len) {
+    const char * p = static_cast<const char *>(b);
+#if defined(__LP64__)
+    const size_t bytes = sizeof(uint64_t);
+#else
+    const size_t bytes = sizeof(uint32_t);
+#endif
+    int align = bytes - reinterpret_cast<uint64_t>(p) % bytes;
+    align = bytes == static_cast<size_t>(align) ? 0 : align;
+
+    if (len < align) {
+        align = len;
+    }
+
+    updateInt64(p, align);
+    p = p + align;
+    len -= align;
+
+    if (len > 0) {
+        assert(0 == reinterpret_cast<uint64_t>(p) % bytes);
+
+        for (int i = len / bytes; i > 0; --i) {
+#if defined(__LP64__)
+            crc = _mm_crc32_u64(crc, *reinterpret_cast<const uint64_t *>(p));
+#else
+            crc = _mm_crc32_u32(crc, *reinterpret_cast<const uint32_t *>(p));
+#endif
+            p = p + bytes;
+        }
+
+        len &= bytes - 1;
+        updateInt64(p, len);
+    }
+}
+
+void HWCrc32c::updateInt64(const char * b, int len) {
+    assert(len < 8);
+
+    switch (len) {
+    case 7:
+        crc = _mm_crc32_u8(crc, *reinterpret_cast<const uint8_t *>(b++));
+
+    case 6:
+        crc = _mm_crc32_u16(crc, *reinterpret_cast<const uint16_t *>(b));
+        b += 2;
+
+        /* case 5 is below: 4 + 1 */
+    case 4:
+        crc = _mm_crc32_u32(crc, *reinterpret_cast<const uint32_t *>(b));
+        break;
+
+    case 3:
+        crc = _mm_crc32_u8(crc, *reinterpret_cast<const uint8_t *>(b++));
+
+    case 2:
+        crc = _mm_crc32_u16(crc, *reinterpret_cast<const uint16_t *>(b));
+        break;
+
+    case 5:
+        crc = _mm_crc32_u32(crc, *reinterpret_cast<const uint32_t *>(b));
+        b += 4;
+
+    case 1:
+        crc = _mm_crc32_u8(crc, *reinterpret_cast<const uint8_t *>(b));
+        break;
+
+    case 0:
+        break;
+    }
+}
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_HWCHECKSUM_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.h
new file mode 100644
index 0000000..d442a5a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/HWCrc32c.h
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_HWCHECKSUM_H_
+#define _HDFS_LIBHDFS3_COMMON_HWCHECKSUM_H_
+
+#include "Checksum.h"
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * Calculate CRC with hardware support.
+ */
+class HWCrc32c: public Checksum {
+public:
+    /**
+     * Constructor.
+     */
+    HWCrc32c() :
+        crc(0xFFFFFFFF) {
+    }
+
+    uint32_t getValue() {
+        return ~crc;
+    }
+
+    /**
+     * @ref Checksum#reset()
+     */
+    void reset() {
+        crc = 0xFFFFFFFF;
+    }
+
+    /**
+     * @ref Checksum#update(const void *, int)
+     */
+    void update(const void *b, int len);
+
+    /**
+     * Destory an HWCrc32 instance.
+     */
+    ~HWCrc32c() {
+    }
+
+    /**
+     * To test if the hardware support this function.
+     * @return true if the hardware support to calculate the CRC.
+     */
+    static bool available();
+
+private:
+    void updateInt64(const char *b, int len);
+
+private:
+    uint32_t crc;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_HWCHECKSUM_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.cc
new file mode 100644
index 0000000..7ca8447
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.cc
@@ -0,0 +1,90 @@
+/**
+ * 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 "Hash.h"
+
+#ifdef NEED_BOOST
+
+#include <boost/functional/hash.hpp>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A hash function object used to hash a boolean value.
+ */
+boost::hash<bool> BoolHasher;
+
+/**
+ * A hash function object used to hash an int value.
+ */
+boost::hash<int> Int32Hasher;
+
+/**
+ * A hash function object used to hash an 64 bit int value.
+ */
+boost::hash<int64_t> Int64Hasher;
+
+/**
+ * A hash function object used to hash a size_t value.
+ */
+boost::hash<size_t> SizeHasher;
+
+/**
+ * A hash function object used to hash a std::string object.
+ */
+boost::hash<std::string> StringHasher;
+}
+}
+
+#else
+
+#include <functional>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A hash function object used to hash a boolean value.
+ */
+std::hash<bool> BoolHasher;
+
+/**
+ * A hash function object used to hash an int value.
+ */
+std::hash<int> Int32Hasher;
+
+/**
+ * A hash function object used to hash an 64 bit int value.
+ */
+std::hash<int64_t> Int64Hasher;
+
+/**
+ * A hash function object used to hash a size_t value.
+ */
+std::hash<size_t> SizeHasher;
+
+/**
+ * A hash function object used to hash a std::string object.
+ */
+std::hash<std::string> StringHasher;
+
+}
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.h
new file mode 100644
index 0000000..079f676
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Hash.h
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_HASH_H_
+#define _HDFS_LIBHDFS3_COMMON_HASH_H_
+
+#include "platform.h"
+
+#include <string>
+#include <vector>
+
+#ifdef NEED_BOOST
+
+#include <boost/functional/hash.hpp>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A hash function object used to hash a boolean value.
+ */
+extern boost::hash<bool> BoolHasher;
+
+/**
+ * A hash function object used to hash an int value.
+ */
+extern boost::hash<int> Int32Hasher;
+
+/**
+ * A hash function object used to hash an 64 bit int value.
+ */
+extern boost::hash<int64_t> Int64Hasher;
+
+/**
+ * A hash function object used to hash a size_t value.
+ */
+extern boost::hash<size_t> SizeHasher;
+
+/**
+ * A hash function object used to hash a std::string object.
+ */
+extern boost::hash<std::string> StringHasher;
+
+}
+}
+
+#define HDFS_HASH_DEFINE(TYPE) \
+    namespace boost{ \
+    template<> \
+    struct hash<TYPE> { \
+        std::size_t operator()(const TYPE & key) const { \
+            return key.hash_value(); \
+        } \
+    }; \
+    }
+
+#else
+
+#include <functional>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A hash function object used to hash a boolean value.
+ */
+extern std::hash<bool> BoolHasher;
+
+/**
+ * A hash function object used to hash an int value.
+ */
+extern std::hash<int> Int32Hasher;
+
+/**
+ * A hash function object used to hash an 64 bit int value.
+ */
+extern std::hash<int64_t> Int64Hasher;
+
+/**
+ * A hash function object used to hash a size_t value.
+ */
+extern std::hash<size_t> SizeHasher;
+
+/**
+ * A hash function object used to hash a std::string object.
+ */
+extern std::hash<std::string> StringHasher;
+
+}
+}
+
+#define HDFS_HASH_DEFINE(TYPE) \
+    namespace std{ \
+    template<> \
+    struct hash<TYPE> { \
+        std::size_t operator()(const TYPE & key) const { \
+            return key.hash_value(); \
+        } \
+    }; \
+    }
+
+#endif
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A hash function used to hash a vector of size_t values.
+ * @param vec The vector's reference which items are to be hashed.
+ * @param size The size of vec.
+ * @return The hash value.
+ * @throw nothrow
+ */
+static inline size_t CombineHasher(const size_t *vec, size_t size) {
+    size_t value = 0;
+
+    for (size_t i = 0; i < size; ++i) {
+        value ^= SizeHasher(vec[i]) << 1;
+    }
+
+    return value;
+}
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_HASH_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.cc
new file mode 100644
index 0000000..d4c885e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.cc
@@ -0,0 +1,109 @@
+/**
+ * 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 "platform.h"
+
+#include "Logger.h"
+
+#include <cassert>
+#include <cstdarg>
+#include <cstdio>
+#include <cstring>
+#include <sstream>
+#include <sys/time.h>
+#include <unistd.h>
+#include <vector>
+
+#include "DateTime.h"
+#include "Thread.h"
+
+namespace hdfs {
+namespace internal {
+
+Logger RootLogger;
+
+static mutex LoggerMutex;
+static THREAD_LOCAL char ProcessId[64];
+
+const char * const SeverityName[] = {
+  "FATAL", "ERROR", "WARNING", "INFO", "DEBUG1", "DEBUG2", "DEBUG3"
+};
+
+static void InitProcessId(char *p, size_t p_len) {
+    std::stringstream ss;
+    ss << "p" << getpid() << ", th" << pthread_self();
+    snprintf(p, p_len, "%s", ss.str().c_str());
+}
+
+Logger::Logger() :
+    fd(STDERR_FILENO), severity(DEFAULT_LOG_LEVEL) {
+}
+
+Logger::~Logger() {
+}
+
+void Logger::setOutputFd(int f) {
+    fd = f;
+}
+
+void Logger::setLogSeverity(LogSeverity l) {
+    severity = l;
+}
+
+void Logger::printf(LogSeverity s, const char *fmt, ...) {
+    va_list ap;
+
+    if (s > severity || fd < 0) {
+        return;
+    }
+
+    try {
+        if (ProcessId[0] == '\0') {
+          InitProcessId(ProcessId, sizeof(ProcessId));
+        }
+        std::vector<char> buffer;
+        struct tm tm_time;
+        struct timeval tval;
+        memset(&tval, 0, sizeof(tval));
+        gettimeofday(&tval, NULL);
+        localtime_r(&tval.tv_sec, &tm_time);
+        //determine buffer size
+        va_start(ap, fmt);
+        int size = vsnprintf(&buffer[0], buffer.size(), fmt, ap);
+        va_end(ap);
+        //100 is enough for prefix
+        buffer.resize(size + 100);
+        size = snprintf(&buffer[0], buffer.size(), "%04d-%02d-%02d %02d:%02d:%02d.%06ld, %s, %s ", tm_time.tm_year + 1900,
+                        1 + tm_time.tm_mon, tm_time.tm_mday, tm_time.tm_hour,
+                        tm_time.tm_min, tm_time.tm_sec, static_cast<long>(tval.tv_usec), ProcessId, SeverityName[s]);
+        va_start(ap, fmt);
+        size += vsnprintf(&buffer[size], buffer.size() - size, fmt, ap);
+        va_end(ap);
+        lock_guard<mutex> lock(LoggerMutex);
+        dprintf(fd, "%s\n", &buffer[0]);
+        return;
+    } catch (const std::exception &e) {
+        dprintf(fd, "%s:%d %s %s", __FILE__, __LINE__,
+                "FATAL: get an unexpected exception:", e.what());
+        throw;
+    }
+}
+
+}
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.h
new file mode 100644
index 0000000..33c1a80
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Logger.h
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_LOGGER_H_
+#define _HDFS_LIBHDFS3_COMMON_LOGGER_H_
+
+#define DEFAULT_LOG_LEVEL INFO
+
+namespace hdfs {
+namespace internal {
+
+extern const char * const SeverityName[];
+
+enum LogSeverity {
+    FATAL, LOG_ERROR, WARNING, INFO, DEBUG1, DEBUG2, DEBUG3, NUM_SEVERITIES
+};
+
+class Logger;
+
+class Logger {
+public:
+    Logger();
+
+    ~Logger();
+
+    void setOutputFd(int f);
+
+    void setLogSeverity(LogSeverity l);
+
+    void printf(LogSeverity s, const char * fmt, ...)
+      __attribute__((format(printf, 3, 4)));
+
+private:
+    int fd;
+    LogSeverity severity;
+};
+
+extern Logger RootLogger;
+
+}
+}
+
+#define LOG(s, fmt, ...) \
+    hdfs::internal::RootLogger.printf(s, fmt, ##__VA_ARGS__)
+
+#endif /* _HDFS_LIBHDFS3_COMMON_LOGGER_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
new file mode 100644
index 0000000..a434aaf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_LRUMAP_H_
+#define _HDFS_LIBHDFS3_COMMON_LRUMAP_H_
+
+#include "Thread.h"
+#include "Unordered.h"
+
+#include <list>
+
+namespace hdfs {
+namespace internal {
+
+template<typename K, typename V>
+class LruMap {
+public:
+    typedef K KeyType;
+    typedef V ValueType;
+    typedef std::pair<K, V> ItmeType;
+    typedef std::list<ItmeType> ListType;
+    typedef unordered_map<K, typename ListType::iterator> MapType;
+
+public:
+    LruMap() :
+        count(0), size(1000) {
+    }
+
+    LruMap(size_t size) :
+        count(0), size(size) {
+    }
+
+    ~LruMap() {
+        lock_guard<mutex> lock(mut);
+        map.clear();
+        list.clear();
+    }
+
+    void resize(size_t s) {
+        lock_guard<mutex> lock(mut);
+        size = s;
+
+        for (size_t i = count; i > s; --i) {
+            map.erase(list.back().first);
+            list.pop_back();
+        }
+    }
+
+    void insert(const KeyType & key, const ValueType & value) {
+        lock_guard<mutex> lock(mut);
+        typename MapType::iterator it = map.find(key);
+
+        if (it != map.end()) {
+            --count;
+            list.erase(it->second);
+        }
+
+        list.push_front(std::make_pair(key, value));
+        map[key] = list.begin();
+        ++count;
+
+        if (count > size) {
+            map.erase(list.back().first);
+            list.pop_back();
+        }
+    }
+
+    void erase(const KeyType & key) {
+        lock_guard<mutex> lock(mut);
+        typename MapType::iterator it = map.find(key);
+
+        if (it != map.end()) {
+            list.erase(it->second);
+            map.erase(it);
+            --count;
+        }
+    }
+
+    bool find(const KeyType & key, ValueType & value) {
+        lock_guard<mutex> lock(mut);
+        typename MapType::iterator it = map.find(key);
+
+        if (it != map.end()) {
+            list.push_front(*(it->second));
+            list.erase(it->second);
+            value = list.front().second;
+            map[key] = list.begin();
+            return true;
+        }
+
+        return false;
+    }
+
+private:
+    size_t count;
+    size_t size;
+    ListType list;
+    MapType map;
+    mutex mut;
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS3_COMMON_LRU_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/MappedFileWrapper.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/MappedFileWrapper.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/MappedFileWrapper.cc
new file mode 100644
index 0000000..18fe995
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/MappedFileWrapper.cc
@@ -0,0 +1,139 @@
+/**
+ * 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 <errno.h>
+#include <fcntl.h>
+#include <limits>
+#include <sstream>
+#include <string>
+#include <sys/mman.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "FileWrapper.h"
+
+namespace hdfs {
+namespace internal {
+
+MappedFileWrapper::MappedFileWrapper() :
+    delegate(true), begin(NULL), position(NULL), fd(-1), size(0) {
+}
+
+MappedFileWrapper::~MappedFileWrapper() {
+    close();
+}
+
+bool MappedFileWrapper::openInternal(int fd, bool delegate, size_t size) {
+    this->delegate = delegate;
+    void *retval = mmap(NULL, size, PROT_READ, MAP_FILE | MAP_PRIVATE, fd, 0);
+    begin = position = static_cast<const char *>(retval);
+
+    if (MAP_FAILED == retval) {
+        begin = position = NULL;
+        close();
+        return false;
+    }
+
+    if (posix_madvise(const_cast<char *>(begin), size, POSIX_MADV_SEQUENTIAL)) {
+        close();
+        return false;
+    }
+
+    return true;
+}
+
+bool MappedFileWrapper::open(int fd, bool delegate) {
+    size = lseek(fd, 0, SEEK_END);
+    lseek(fd, 0, SEEK_SET);
+    std::stringstream ss;
+    ss << "FileDescriptor " << fd;
+    path = ss.str();
+
+    if (static_cast<uint64_t>(size) >
+        static_cast<uint64_t>(std::numeric_limits<size_t>::max())) {
+        THROW(HdfsIOException,
+              "Cannot create memory mapped file for \"%s\", file is too large.",
+              path.c_str());
+    }
+
+    return openInternal(fd, delegate, static_cast<size_t>(size));
+}
+
+bool MappedFileWrapper::open(const std::string &path) {
+    struct stat st;
+
+    if (stat(path.c_str(), &st)) {
+        return false;
+    }
+
+    size = st.st_size;
+
+    if (static_cast<uint64_t>(size) >
+        static_cast<uint64_t>(std::numeric_limits<size_t>::max())) {
+        THROW(HdfsIOException,
+              "Cannot create memory mapped file for \"%s\", file is too large.",
+              path.c_str());
+    }
+
+    fd = ::open(path.c_str(), O_RDONLY);
+
+    if (fd < 0) {
+        return false;
+    }
+
+    this->path = path;
+    return openInternal(fd, true, st.st_size);
+}
+
+void MappedFileWrapper::close() {
+    if (NULL != begin) {
+        ::munmap(const_cast<char *>(begin), static_cast<size_t>(size));
+        begin = position = NULL;
+    }
+    if (fd >= 0 && delegate) {
+        ::close(fd);
+    }
+
+    fd = -1;
+    size = 0;
+    delegate = true;
+    path.clear();
+}
+
+const char * MappedFileWrapper::read(std::vector<char> &buffer, int32_t size) {
+    assert(NULL != begin && NULL != position);
+    const char * retval = position;
+    position += size;
+    return retval;
+}
+
+void MappedFileWrapper::copy(char *buffer, int32_t size) {
+    assert(NULL != begin && NULL != position);
+    memcpy(buffer, position, size);
+    position += size;
+}
+
+void MappedFileWrapper::seek(int64_t offset) {
+    assert(NULL != begin && NULL != position);
+    position = begin + offset;
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.cc
new file mode 100644
index 0000000..bbba4a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.cc
@@ -0,0 +1,96 @@
+/**
+ * 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 "SWCrc32c.h"
+
+namespace hdfs {
+namespace internal {
+
+/*
+ * The following CRC lookup table was generated automagically
+ * using the following model parameters:
+ *
+ * Generator Polynomial = ................. 0x1EDC6F41
+ * Generator Polynomial Length = .......... 32 bits
+ * Reflected Bits = ....................... TRUE
+ * Table Generation Offset = .............. 32 bits
+ * Number of Slices = ..................... 8 slices
+ * Slice Lengths = ........................ 8 8 8 8 8 8 8 8
+ */
+
+static const uint32_t crc_tableil8_o32[256] = { 0x00000000, 0xF26B8303,
+                                                0xE13B70F7, 0x1350F3F4, 0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB,
+                                                0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, 0x4D43CFD0, 0xBF284CD3,
+                                                0xAC78BF27, 0x5E133C24, 0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B,
+                                                0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, 0x9A879FA0, 0x68EC1CA3,
+                                                0x7BBCEF57, 0x89D76C54, 0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B,
+                                                0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, 0xE72719C1, 0x154C9AC2,
+                                                0x061C6936, 0xF477EA35, 0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5,
+                                                0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, 0x30E349B1, 0xC288CAB2,
+                                                0xD1D83946, 0x23B3BA45, 0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A,
+                                                0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, 0x7DA08661, 0x8FCB0562,
+                                                0x9C9BF696, 0x6EF07595, 0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48,
+                                                0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, 0xCBA24573, 0x39C9C670,
+                                                0x2A993584, 0xD8F2B687, 0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198,
+                                                0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, 0x96BF4DCC, 0x64D4CECF,
+                                                0x77843D3B, 0x85EFBE38, 0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8,
+                                                0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, 0x61C69362, 0x93AD1061,
+                                                0x80FDE395, 0x72966096, 0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789,
+                                                0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, 0x2C855CB2, 0xDEEEDFB1,
+                                                0xCDBE2C45, 0x3FD5AF46, 0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9,
+                                                0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, 0xFB410CC2, 0x092A8FC1,
+                                                0x1A7A7C35, 0xE811FF36, 0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829,
+                                                0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, 0x456CAC67, 0xB7072F64,
+                                                0xA457DC90, 0x563C5F93, 0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043,
+                                                0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, 0x92A8FC17, 0x60C37F14,
+                                                0x73938CE0, 0x81F80FE3, 0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC,
+                                                0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, 0xDFEB33C7, 0x2D80B0C4,
+                                                0x3ED04330, 0xCCBBC033, 0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652,
+                                                0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, 0x2892ED69, 0xDAF96E6A,
+                                                0xC9A99D9E, 0x3BC21E9D, 0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982,
+                                                0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, 0x758FE5D6, 0x87E466D5,
+                                                0x94B49521, 0x66DF1622, 0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2,
+                                                0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, 0xC38D26C4, 0x31E6A5C7,
+                                                0x22B65633, 0xD0DDD530, 0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F,
+                                                0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, 0x8ECEE914, 0x7CA56A17,
+                                                0x6FF599E3, 0x9D9E1AE0, 0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F,
+                                                0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, 0x590AB964, 0xAB613A67,
+                                                0xB831C993, 0x4A5A4A90, 0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F,
+                                                0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, 0x24AA3F05, 0xD6C1BC06,
+                                                0xC5914FF2, 0x37FACCF1, 0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321,
+                                                0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, 0xF36E6F75, 0x0105EC76,
+                                                0x12551F82, 0xE03E9C81, 0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E,
+                                                0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, 0xBE2DA0A5, 0x4C4623A6,
+                                                0x5F16D052, 0xAD7D5351
+                                              };
+
+/*
+ * end of the CRC lookup table crc_tableil8_o32
+ */
+
+void SWCrc32c::update(const void * b, int len) {
+    const char * p = static_cast<const char *>(b);
+    const char * e = p + len;
+
+    while (p < e) {
+        crc = crc_tableil8_o32[(crc ^ *p++) & 0x000000FF] ^ (crc >> 8);
+    }
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d873425a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.h
new file mode 100644
index 0000000..8d21064
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SWCrc32c.h
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_COMMON_SWCRC32C_H_
+#define _HDFS_LIBHDFS3_COMMON_SWCRC32C_H_
+
+#include "Checksum.h"
+#include "platform.h"
+
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+class SWCrc32c: public Checksum {
+public:
+    SWCrc32c() :
+        crc(0xFFFFFFFF) {
+    }
+
+    uint32_t getValue() {
+        return ~crc;
+    }
+
+    void reset() {
+        crc = 0xFFFFFFFF;
+    }
+
+    void update(const void *b, int len);
+
+    ~SWCrc32c() {
+    }
+
+private:
+    uint32_t crc;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_SWCRC32C_H_ */