http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/Thread.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/Thread.cpp 
b/depends/libhdfs3/src/common/Thread.cpp
new file mode 100644
index 0000000..14cf217
--- /dev/null
+++ b/depends/libhdfs3/src/common/Thread.cpp
@@ -0,0 +1,54 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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 <unistd.h>
+
+#include "Thread.h"
+
+namespace Hdfs {
+namespace Internal {
+
+sigset_t ThreadBlockSignal() {
+    sigset_t sigs;
+    sigset_t oldMask;
+    sigemptyset(&sigs);
+    sigaddset(&sigs, SIGHUP);
+    sigaddset(&sigs, SIGINT);
+    sigaddset(&sigs, SIGTERM);
+    sigaddset(&sigs, SIGUSR1);
+    sigaddset(&sigs, SIGUSR2);
+    pthread_sigmask(SIG_BLOCK, &sigs, &oldMask);
+    return oldMask;
+}
+
+void ThreadUnBlockSignal(sigset_t sigs) {
+    pthread_sigmask(SIG_SETMASK, &sigs, 0);
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/Thread.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/Thread.h 
b/depends/libhdfs3/src/common/Thread.h
new file mode 100644
index 0000000..4c2401e
--- /dev/null
+++ b/depends/libhdfs3/src/common/Thread.h
@@ -0,0 +1,107 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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_THREAD_H_
+#define _HDFS_LIBHDFS3_COMMON_THREAD_H_
+
+#include "platform.h"
+
+#include <signal.h>
+
+#ifdef NEED_BOOST
+
+#include <boost/thread.hpp>
+
+namespace Hdfs {
+namespace Internal {
+
+using boost::thread;
+using boost::mutex;
+using boost::lock_guard;
+using boost::unique_lock;
+using boost::condition_variable;
+using boost::defer_lock_t;
+using boost::once_flag;
+using boost::call_once;
+using namespace boost::this_thread;
+
+}
+}
+
+#else
+
+#include <thread>
+#include <mutex>
+#include <condition_variable>
+
+namespace Hdfs {
+namespace Internal {
+
+using std::thread;
+using std::mutex;
+using std::lock_guard;
+using std::unique_lock;
+using std::condition_variable;
+using std::defer_lock_t;
+using std::once_flag;
+using std::call_once;
+using namespace std::this_thread;
+
+}
+}
+#endif
+
+namespace Hdfs {
+namespace Internal {
+
+/*
+ * make the background thread ignore these signals (which should allow that
+ * they be delivered to the main thread)
+ */
+sigset_t ThreadBlockSignal();
+
+/*
+ * Restore previous signals.
+ */
+void ThreadUnBlockSignal(sigset_t sigs);
+
+}
+}
+
+#define CREATE_THREAD(retval, fun) \
+    do { \
+        sigset_t sigs = Hdfs::Internal::ThreadBlockSignal(); \
+        try { \
+            retval = Hdfs::Internal::thread(fun); \
+            Hdfs::Internal::ThreadUnBlockSignal(sigs); \
+        } catch (...) { \
+            Hdfs::Internal::ThreadUnBlockSignal(sigs); \
+            throw; \
+        } \
+    } while(0)
+
+#endif /* _HDFS_LIBHDFS3_COMMON_THREAD_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/Unordered.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/Unordered.h 
b/depends/libhdfs3/src/common/Unordered.h
new file mode 100644
index 0000000..fa008ed
--- /dev/null
+++ b/depends/libhdfs3/src/common/Unordered.h
@@ -0,0 +1,62 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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_UNORDEREDMAP_H_
+#define _HDFS_LIBHDFS3_COMMON_UNORDEREDMAP_H_
+
+#include "platform.h"
+
+#ifdef NEED_BOOST
+
+#include <boost/unordered_map.hpp>
+#include <boost/unordered_set.hpp>
+
+namespace Hdfs {
+namespace Internal {
+
+using boost::unordered_map;
+using boost::unordered_set;
+
+}
+}
+
+#else
+
+#include <unordered_map>
+#include <unordered_set>
+
+namespace Hdfs {
+namespace Internal {
+
+using std::unordered_map;
+using std::unordered_set;
+
+}
+}
+#endif
+
+#endif /* _HDFS_LIBHDFS3_COMMON_UNORDEREDMAP_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/WritableUtils.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/WritableUtils.cpp 
b/depends/libhdfs3/src/common/WritableUtils.cpp
new file mode 100644
index 0000000..71523e4
--- /dev/null
+++ b/depends/libhdfs3/src/common/WritableUtils.cpp
@@ -0,0 +1,194 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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 <stdexcept>
+#include <limits>
+#include <cstring>
+#include <string>
+#include <arpa/inet.h>
+
+#include "WritableUtils.h"
+
+namespace Hdfs {
+namespace Internal {
+
+WritableUtils::WritableUtils(char * b, size_t l) :
+    buffer(b), len(l), current(0) {
+}
+
+int32_t WritableUtils::ReadInt32() {
+    int64_t val;
+    val = ReadInt64();
+
+    if (val < std::numeric_limits<int32_t>::min()
+            || val > std::numeric_limits<int32_t>::max()) {
+        throw std::range_error("overflow");
+    }
+
+    return val;
+}
+
+int64_t WritableUtils::ReadInt64() {
+    int64_t value;
+    int firstByte = readByte();
+    int len = decodeWritableUtilsSize(firstByte);
+
+    if (len == 1) {
+        value = firstByte;
+        return value;
+    }
+
+    long i = 0;
+
+    for (int idx = 0; idx < len - 1; idx++) {
+        unsigned char b = readByte();
+        i = i << 8;
+        i = i | (b & 0xFF);
+    }
+
+    value = (isNegativeWritableUtils(firstByte) ? (i ^ -1L) : i);
+    return value;
+}
+
+void WritableUtils::ReadRaw(char * buf, size_t size) {
+    if (size > len - current) {
+        throw std::range_error("overflow");
+    }
+
+    memcpy(buf, buffer + current, size);
+    current += size;
+}
+
+std::string WritableUtils::ReadText() {
+    int32_t length;
+    std::string retval;
+    length = ReadInt32();
+    retval.resize(length);
+    ReadRaw(&retval[0], length);
+    return retval;
+}
+
+size_t WritableUtils::WriteInt32(int32_t value) {
+    return WriteInt64(value);
+}
+
+size_t WritableUtils::WriteInt64(int64_t value) {
+    size_t retval = 1;
+
+    if (value >= -112 && value <= 127) {
+        writeByte((int) value);
+        return retval;
+    }
+
+    int len = -112;
+
+    if (value < 0) {
+        value ^= -1L; // take one's complement'
+        len = -120;
+    }
+
+    long tmp = value;
+
+    while (tmp != 0) {
+        tmp = tmp >> 8;
+        len--;
+    }
+
+    ++retval;
+    writeByte((int) len);
+    len = (len < -120) ? -(len + 120) : -(len + 112);
+
+    for (int idx = len; idx != 0; idx--) {
+        int shiftbits = (idx - 1) * 8;
+        long mask = 0xFFL << shiftbits;
+        ++retval;
+        writeByte((int)((value & mask) >> shiftbits));
+    }
+
+    return retval;
+}
+
+size_t WritableUtils::WriteRaw(const char * buf, size_t size) {
+    if (size > len - current) {
+        throw std::range_error("overflow");
+    }
+
+    memcpy(buffer + current, buf, size);
+    current += size;
+    return size;
+}
+
+int WritableUtils::decodeWritableUtilsSize(int value) {
+    if (value >= -112) {
+        return 1;
+    } else if (value < -120) {
+        return -119 - value;
+    }
+
+    return -111 - value;
+}
+
+int WritableUtils::readByte() {
+    if (sizeof(char) > len - current) {
+        throw std::range_error("overflow");
+    }
+
+    return buffer[current++];
+}
+
+void WritableUtils::writeByte(int val) {
+    if (sizeof(char) > len - current) {
+        throw std::range_error("overflow");
+    }
+
+    buffer[current++] = val;
+}
+
+size_t WritableUtils::WriteText(const std::string & str) {
+    size_t retval = 0;
+    int32_t length = str.length();
+    retval += WriteInt32(length);
+    retval += WriteRaw(&str[0], length);
+    return retval;
+}
+
+bool WritableUtils::isNegativeWritableUtils(int value) {
+    return value < -120 || (value >= -112 && value < 0);
+}
+
+int32_t WritableUtils::ReadBigEndian32() {
+    char buf[sizeof(int32_t)];
+
+    for (size_t i = 0; i < sizeof(int32_t); ++i) {
+        buf[i] = readByte();
+    }
+
+    return ntohl(*(uint32_t *) buf);
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/WritableUtils.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/WritableUtils.h 
b/depends/libhdfs3/src/common/WritableUtils.h
new file mode 100644
index 0000000..c40469d
--- /dev/null
+++ b/depends/libhdfs3/src/common/WritableUtils.h
@@ -0,0 +1,75 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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_LIBHDFS_3_UTIL_WritableUtils_H_
+#define _HDFS_LIBHDFS_3_UTIL_WritableUtils_H_
+
+#include <string>
+
+namespace Hdfs {
+namespace Internal {
+
+class WritableUtils {
+public:
+    WritableUtils(char * b, size_t l);
+
+    int32_t ReadInt32();
+
+    int64_t ReadInt64();
+
+    void ReadRaw(char * buf, size_t size);
+
+    std::string ReadText();
+
+    int readByte();
+
+    size_t WriteInt32(int32_t value);
+
+    size_t WriteInt64(int64_t value);
+
+    size_t WriteRaw(const char * buf, size_t size);
+
+    size_t WriteText(const std::string & str);
+
+private:
+    int decodeWritableUtilsSize(int value);
+
+    void writeByte(int val);
+
+    bool isNegativeWritableUtils(int value);
+
+    int32_t ReadBigEndian32();
+
+private:
+    char * buffer;
+    size_t len;
+    size_t current;
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS_3_UTIL_WritableUtils_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/WriteBuffer.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/WriteBuffer.cpp 
b/depends/libhdfs3/src/common/WriteBuffer.cpp
new file mode 100644
index 0000000..fde8f77
--- /dev/null
+++ b/depends/libhdfs3/src/common/WriteBuffer.cpp
@@ -0,0 +1,80 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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 "WriteBuffer.h"
+
+#include <google/protobuf/io/coded_stream.h>
+
+using namespace google::protobuf::io;
+using google::protobuf::uint8;
+
+namespace Hdfs {
+namespace Internal {
+
+#define WRITEBUFFER_INIT_SIZE 64
+
+WriteBuffer::WriteBuffer() :
+    size(0), buffer(WRITEBUFFER_INIT_SIZE) {
+}
+
+WriteBuffer::~WriteBuffer() {
+}
+
+void WriteBuffer::writeVarint32(int32_t value, size_t pos) {
+    char buffer[5];
+    uint8 * end = CodedOutputStream::WriteVarint32ToArray(value,
+                  reinterpret_cast<uint8 *>(buffer));
+    write(buffer, reinterpret_cast<char *>(end) - buffer, pos);
+}
+
+char * WriteBuffer::alloc(size_t offset, size_t s) {
+    assert(offset <= size && size <= buffer.size());
+
+    if (offset > size) {
+        return NULL;
+    }
+
+    size_t target = offset + s;
+
+    if (target >= buffer.size()) {
+        target = target > 2 * buffer.size() ? target : 2 * buffer.size();
+        buffer.resize(target);
+    }
+
+    size = offset + s;
+    return &buffer[offset];
+}
+
+void WriteBuffer::write(const void * bytes, size_t s, size_t pos) {
+    assert(NULL != bytes);
+    assert(pos <= size && pos < buffer.size());
+    char * p = alloc(size, s);
+    memcpy(p, bytes, s);
+}
+
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/WriteBuffer.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/WriteBuffer.h 
b/depends/libhdfs3/src/common/WriteBuffer.h
new file mode 100644
index 0000000..9cae81b
--- /dev/null
+++ b/depends/libhdfs3/src/common/WriteBuffer.h
@@ -0,0 +1,245 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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_WRITEBUFFER_H_
+#define _HDFS_LIBHDFS3_COMMON_WRITEBUFFER_H_
+
+#include <cassert>
+#include <cstddef>
+#include <cstring>
+#include <stdint.h>
+#include <vector>
+
+#include <arpa/inet.h>
+
+namespace Hdfs {
+namespace Internal {
+
+/**
+ * a data buffer used to read and write.
+ */
+class WriteBuffer {
+public:
+    /**
+     * Construct a empty buffer.
+     * @throw nothrow
+     */
+    WriteBuffer();
+
+    /**
+     * Destroy a buffer.
+     * @throw nothrow
+     */
+    ~WriteBuffer();
+
+    /**
+     * Write string into buffer.
+     * Terminated '\0' will also be written into buffer.
+     * @param str The string to be written.
+     * @throw nothrow
+     */
+    void writeString(const char * str) {
+        writeString(str, size);
+    }
+
+    /**
+     * Write string into buffer with given position.
+     * Terminated '\0' will also be written into buffer and the data after 
given position will be overwritten.
+     * @param str The string to be written.
+     * @param pos The given start position in buffer.
+     * @throw nothrow
+     */
+    void writeString(const char * str, size_t pos) {
+        write(str, strlen(str) + 1, pos);
+    }
+
+    /**
+     * Write a vector into buffer.
+     * @param bytes The data be written.
+     * @param s The size of data.
+     */
+    void write(const void * bytes, size_t s) {
+        write(bytes, s, size);
+    }
+
+    /**
+     * Write a vector into buffer with given position.
+     * The data after given position will be overwritten.
+     * @param bytes The data be written.
+     * @param s The size of data.
+     * @param pos The given start position in buffer.
+     */
+    void write(const void * bytes, size_t s, size_t pos);
+
+    /**
+     * Write char into buffer.
+     * @param value The char to be written.
+     * @throw nothrow
+     */
+    void write(char value) {
+        write(value, size);
+    }
+
+    /**
+     * Write char into buffer with given position.
+     * The data after given position will be overwritten.
+     * @param value The char to be written.
+     * @param pos The given start position in buffer.
+     * @throw nothrow
+     */
+    void write(char value, size_t pos) {
+        write(&value, sizeof(value));
+    }
+
+    /**
+     * Convert the 16 bit integer into big endian and write into buffer.
+     * @param value The integer to be written.
+     * @throw nothrow
+     */
+    void writeBigEndian(int16_t value) {
+        writeBigEndian(value, size);
+    }
+
+    /**
+     * Convert the 16 bit integer into big endian and write into buffer with 
given position.
+     * The data after given position will be overwritten.
+     * @param value The integer to be written.
+     * @param pos The given start position in buffer.
+     * @throw nothrow
+     */
+    void writeBigEndian(int16_t value, size_t pos) {
+        int16_t v = htons(value);
+        write((const char *) &v, sizeof(v));
+    }
+
+    /**
+     * Convert the 32 bit integer into big endian and write into buffer.
+     * @param value The integer to be written.
+     * @throw nothrow
+     */
+    void writeBigEndian(int32_t value) {
+        writeBigEndian(value, size);
+    }
+
+    /**
+     * Convert the 32 bit integer into big endian and write into buffer with 
given position.
+     * The data after given position will be overwritten.
+     * @param value The integer to be written.
+     * @param pos The given start position in buffer.
+     * @throw nothrow
+     */
+    void writeBigEndian(int32_t value, size_t pos) {
+        int32_t v = htonl(value);
+        write((const char *) &v, sizeof(v), pos);
+    }
+
+    /**
+     * Convert the 32 bit integer into varint and write into buffer.
+     * @param value The integer to be written.
+     * @throw nothrow
+     */
+    void writeVarint32(int32_t value) {
+        writeVarint32(value, size);
+    }
+
+    /**
+     * Convert the 32 bit integer into varint and write into buffer with given 
position.
+     * The data after given position will be overwritten.
+     * @param value The integer to be written.
+     * @param pos The given start position in buffer.
+     * @throw nothrow
+     */
+    void writeVarint32(int32_t value, size_t pos);
+
+    /**
+     * Get the buffered data from given offset.
+     * @param offset The size of bytes to be ignored from begin of buffer.
+     * @return The buffered data, or NULL if offset is over the end of data.
+     * @throw nothrow
+     */
+    const char * getBuffer(size_t offset) const {
+        assert(offset <= size && offset < buffer.size());
+
+        if (offset >= size) {
+            return NULL;
+        }
+
+        return &buffer[offset];
+    }
+
+    /**
+     * Get the total bytes in the buffer from offset.
+     * @param offset The size of bytes to be ignored from begin of buffer.
+     * @return The total bytes in the buffer from offset.
+     * @throw nothrow
+     */
+    size_t getDataSize(size_t offset) const {
+        assert(offset <= size);
+        return size - offset;
+    }
+
+    /**
+     * Allocate a region of buffer to caller.
+     * Caller should copy the data into this region manually instead of 
calling Buffer's method.
+     *      This method will set the current data size to offset + s, caller 
may need to reset it to correct value.
+     * @param offset Expected offset in the buffer, the data after given 
offset will be overwritten.
+     * @param s Allocate the size of byte.
+     * @return The start address in the buffer from offset, or NULL if offset 
is over the end of data.
+     * @throw nothrow
+     */
+    char * alloc(size_t offset, size_t s);
+
+    /**
+     * Allocate a region of buffer to caller from the end of current buffer.
+     * Caller should copy the data into this region manually instead of 
calling Buffer's method.
+     *      This method will set the current data size to size + s, caller may 
need to reset it to correct value.
+     * @param s Allocate the size of byte.
+     * @return The start address in the buffer from offset.
+     * @throw nothrow
+     */
+    char * alloc(size_t s) {
+        return alloc(size, s);
+    }
+
+    /**
+     * Set the available data size.
+     * @param s The size to be set.
+     * throw nothrow
+     */
+    void setBufferDataSize(size_t s) {
+        size = s;
+    }
+
+private:
+    size_t size; //current write position.
+    std::vector<char> buffer;
+
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS3_COMMON_WRITEBUFFER_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/XmlConfig.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/XmlConfig.cpp 
b/depends/libhdfs3/src/common/XmlConfig.cpp
new file mode 100644
index 0000000..dd9951c
--- /dev/null
+++ b/depends/libhdfs3/src/common/XmlConfig.cpp
@@ -0,0 +1,397 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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 "ExceptionInternal.h"
+#include "XmlConfig.h"
+#include "Hash.h"
+
+#include <cassert>
+#include <errno.h>
+#include <fstream>
+#include <libxml/parser.h>
+#include <libxml/tree.h>
+#include <limits>
+#include <string.h>
+#include <unistd.h>
+
+using namespace Hdfs::Internal;
+
+namespace Hdfs {
+
+typedef std::map<std::string, std::string>::const_iterator Iterator;
+typedef std::map<std::string, std::string> Map;
+
+static int32_t StrToInt32(const char * str) {
+    long retval;
+    char * end = NULL;
+    errno = 0;
+    retval = strtol(str, &end, 0);
+
+    if (EINVAL == errno || 0 != *end) {
+        THROW(HdfsBadNumFoumat, "Invalid int32_t type: %s", str);
+    }
+
+    if (ERANGE == errno || retval > std::numeric_limits<int32_t>::max()
+            || retval < std::numeric_limits<int32_t>::min()) {
+        THROW(HdfsBadNumFoumat, "Underflow/Overflow int32_t type: %s", str);
+    }
+
+    return retval;
+}
+
+static int64_t StrToInt64(const char * str) {
+    long long retval;
+    char * end = NULL;
+    errno = 0;
+    retval = strtoll(str, &end, 0);
+
+    if (EINVAL == errno || 0 != *end) {
+        THROW(HdfsBadNumFoumat, "Invalid int64_t type: %s", str);
+    }
+
+    if (ERANGE == errno || retval > std::numeric_limits<int64_t>::max()
+            || retval < std::numeric_limits<int64_t>::min()) {
+        THROW(HdfsBadNumFoumat, "Underflow/Overflow int64_t type: %s", str);
+    }
+
+    return retval;
+}
+
+static bool StrToBool(const char * str) {
+    bool retval = false;
+
+    if (!strcasecmp(str, "true") || !strcmp(str, "1")) {
+        retval = true;
+    } else if (!strcasecmp(str, "false") || !strcmp(str, "0")) {
+        retval = false;
+    } else {
+        THROW(HdfsBadBoolFoumat, "Invalid bool type: %s", str);
+    }
+
+    return retval;
+}
+
+static double StrToDouble(const char * str) {
+    double retval;
+    char * end = NULL;
+    errno = 0;
+    retval = strtod(str, &end);
+
+    if (EINVAL == errno || 0 != *end) {
+        THROW(HdfsBadNumFoumat, "Invalid double type: %s", str);
+    }
+
+    if (ERANGE == errno || retval > std::numeric_limits<double>::max()
+            || retval < std::numeric_limits<double>::min()) {
+        THROW(HdfsBadNumFoumat, "Underflow/Overflow int64_t type: %s", str);
+    }
+
+    return retval;
+}
+
+static void readConfigItem(xmlNodePtr root, Map & kv, const char * path) {
+    std::string key, value;
+    xmlNodePtr curNode;
+    bool hasname = false, hasvalue = false;
+
+    for (curNode = root; NULL != curNode; curNode = curNode->next) {
+        if (curNode->type != XML_ELEMENT_NODE) {
+            continue;
+        }
+
+        if (!hasname && !strcmp((const char *) curNode->name, "name")) {
+            if (NULL != curNode->children
+                    && XML_TEXT_NODE == curNode->children->type) {
+                key = (const char *) curNode->children->content;
+                hasname = true;
+            }
+        } else if (!hasvalue
+                   && !strcmp((const char *) curNode->name, "value")) {
+            if (NULL != curNode->children
+                    && XML_TEXT_NODE == curNode->children->type) {
+                value = (const char *) curNode->children->content;
+                hasvalue = true;
+            }
+        } else {
+            continue;
+        }
+    }
+
+    if (hasname && hasvalue) {
+        kv[key] = value;
+        return;
+    } else if (hasname) {
+        kv[key] = "";
+        return;
+    }
+
+    THROW(HdfsBadConfigFoumat, "Config cannot parse configure file: \"%s\"",
+          path);
+}
+
+static void readConfigItems(xmlDocPtr doc, Map & kv, const char * path) {
+    xmlNodePtr root, curNode;
+    root = xmlDocGetRootElement(doc);
+
+    if (NULL == root || strcmp((const char *) root->name, "configuration")) {
+        THROW(HdfsBadConfigFoumat, "Config cannot parse configure file: 
\"%s\"",
+              path);
+    }
+
+    /*
+     * for each property
+     */
+    for (curNode = root->children; NULL != curNode; curNode = curNode->next) {
+        if (curNode->type != XML_ELEMENT_NODE) {
+            continue;
+        }
+
+        if (strcmp((const char *) curNode->name, "property")) {
+            THROW(HdfsBadConfigFoumat,
+                  "Config cannot parse configure file: \"%s\"", path);
+        }
+
+        readConfigItem(curNode->children, kv, path);
+    }
+}
+
+Config::Config(const char * p) :
+    path(p) {
+    update(p);
+}
+
+void Config::update(const char * p) {
+    xmlDocPtr doc; /* the resulting document tree */
+    LIBXML_TEST_VERSION
+    kv.clear();
+    path = p;
+
+    if (access(path.c_str(), R_OK)) {
+        THROW(HdfsBadConfigFoumat, "Cannot read configure file: \"%s\", %s",
+              path.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    /* parse the file */
+    doc = xmlReadFile(path.c_str(), NULL, 0);
+
+    try {
+        /* check if parsing succeeded */
+        if (doc == NULL) {
+            THROW(HdfsBadConfigFoumat,
+                  "Config cannot parse configure file: \"%s\"", path.c_str());
+        } else {
+            readConfigItems(doc, kv, path.c_str());
+            /* free up the resulting document */
+            xmlFreeDoc(doc);
+        }
+    } catch (...) {
+        xmlFreeDoc(doc);
+        throw;
+    }
+}
+
+const char * Config::getString(const char * key) const {
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return it->second.c_str();
+}
+
+const char * Config::getString(const char * key, const char * def) const {
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        return def;
+    } else {
+        return it->second.c_str();
+    }
+}
+
+const char * Config::getString(const std::string & key) const {
+    return getString(key.c_str());
+}
+
+const char * Config::getString(const std::string & key,
+                               const std::string & def) const {
+    return getString(key.c_str(), def.c_str());
+}
+
+int64_t Config::getInt64(const char * key) const {
+    int64_t retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    try {
+        retval = StrToInt64(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+int64_t Config::getInt64(const char * key, int64_t def) const {
+    int64_t retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        return def;
+    }
+
+    try {
+        retval = StrToInt64(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+int32_t Config::getInt32(const char * key) const {
+    int32_t retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    try {
+        retval = StrToInt32(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+int32_t Config::getInt32(const char * key, int32_t def) const {
+    int32_t retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        return def;
+    }
+
+    try {
+        retval = StrToInt32(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+double Config::getDouble(const char * key) const {
+    double retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    try {
+        retval = StrToDouble(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+double Config::getDouble(const char * key, double def) const {
+    double retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        return def;
+    }
+
+    try {
+        retval = StrToDouble(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+bool Config::getBool(const char * key) const {
+    bool retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    try {
+        retval = StrToBool(it->second.c_str());
+    } catch (const HdfsBadBoolFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+bool Config::getBool(const char * key, bool def) const {
+    bool retval;
+    Iterator it = kv.find(key);
+
+    if (kv.end() == it) {
+        return def;
+    }
+
+    try {
+        retval = StrToBool(it->second.c_str());
+    } catch (const HdfsBadNumFoumat & e) {
+        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
+    }
+
+    return retval;
+}
+
+size_t Config::hash_value() const {
+    std::vector<size_t> values;
+    std::map<std::string, std::string>::const_iterator s, e;
+    e = kv.end();
+
+    for (s = kv.begin(); s != e; ++s) {
+        values.push_back(StringHasher(s->first));
+        values.push_back(StringHasher(s->second));
+    }
+
+    return CombineHasher(&values[0], values.size());
+}
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/bc0904ab/depends/libhdfs3/src/common/XmlConfig.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/XmlConfig.h 
b/depends/libhdfs3/src/common/XmlConfig.h
new file mode 100644
index 0000000..8a4a3c3
--- /dev/null
+++ b/depends/libhdfs3/src/common/XmlConfig.h
@@ -0,0 +1,192 @@
+/********************************************************************
+ * Copyright (c) 2013 - 2014, Pivotal Inc.
+ * All rights reserved.
+ *
+ * Author: Zhanwei Wang
+ ********************************************************************/
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.0
+ ********************************************************************/
+/**
+ * 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_XMLCONFIG_H_
+#define _HDFS_LIBHDFS3_COMMON_XMLCONFIG_H_
+
+#include <stdint.h>
+#include <string>
+#include <sstream>
+#include <map>
+
+namespace Hdfs {
+
+/**
+ * A configure file parser.
+ */
+class Config {
+public:
+    /**
+     * Construct a empty Config instance.
+     */
+    Config() {
+    }
+
+    /**
+     * Construct a Config with given configure file.
+     * @param path The path of configure file.
+     * @throw HdfsBadConfigFoumat
+     */
+    Config(const char * path);
+
+    /**
+     * Parse the configure file.
+     * @throw HdfsBadConfigFoumat
+     */
+    void update(const char * path);
+
+    /**
+     * Get a string with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    const char * getString(const char * key) const;
+
+    /**
+     * Get a string with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The defalut value.
+     * @return The value of configure item.
+     */
+    const char * getString(const char * key, const char * def) const;
+
+    /**
+     * Get a string with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    const char * getString(const std::string & key) const;
+
+    /**
+     * Get a string with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The defalut value.
+     * @return The value of configure item.
+     */
+    const char * getString(const std::string & key,
+                           const std::string & def) const;
+
+    /**
+     * Get a 64 bit integer with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    int64_t getInt64(const char * key) const;
+
+    /**
+     * Get a 64 bit integer with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The defalut value.
+     * @return The value of configure item.
+     */
+    int64_t getInt64(const char * key, int64_t def) const;
+
+    /**
+     * Get a 32 bit integer with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    int32_t getInt32(const char * key) const;
+
+    /**
+     * Get a 32 bit integer with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The defalut value.
+     * @return The value of configure item.
+     */
+    int32_t getInt32(const char * key, int32_t def) const;
+
+    /**
+     * Get a double with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    double getDouble(const char * key) const;
+
+    /**
+     * Get a double with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The defalut value.
+     * @return The value of configure item.
+     */
+    double getDouble(const char * key, double def) const;
+
+    /**
+     * Get a boolean with given configure key.
+     * @param key The key of the configure item.
+     * @return The value of configure item.
+     * @throw HdfsConfigNotFound
+     */
+    bool getBool(const char * key) const;
+
+    /**
+     * Get a boolean with given configure key.
+     * Return the default value def if key is not found.
+     * @param key The key of the configure item.
+     * @param def The default value.
+     * @return The value of configure item.
+     */
+    bool getBool(const char * key, bool def) const;
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    template<typename T>
+    void set(const char * key, T const & value) {
+        std::stringstream ss;
+        ss.imbue(std::locale::classic());
+        ss << value;
+        kv[key] = ss.str();
+    }
+
+    /**
+     * Get the hash value of this object
+     *
+     * @return The hash value
+     */
+    size_t hash_value() const;
+
+private:
+    std::string path;
+    std::map<std::string, std::string> kv;
+};
+
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_XMLCONFIG_H_ */


Reply via email to