Просмотр исходного кода

HADOOP-10389. Native RPCv9 client (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HADOOP-10388@1591534 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 лет назад
Родитель
Сommit
a7d4a7aed9

+ 122 - 0
hadoop-native-core/CMakeLists.txt

@@ -0,0 +1,122 @@
+cmake_minimum_required(VERSION 2.6)
+set(CMAKE_BUILD_TYPE, Release) # Default to release builds
+enable_testing()
+MESSAGE(STATUS "Building hadoop-native-core, the native Hadoop core libraries.")
+
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -Wextra -O2 -fno-strict-aliasing")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
+set(CMAKE_CXX_FLAGS "${CMAKE_C_FLAGS} -Wno-unused")
+
+macro(add_utest utest)
+    add_test(${utest} ${CMAKE_CURRENT_BINARY_DIR}/${utest} ${utest})
+endmacro(add_utest)
+
+# Find libuv
+find_library(LIBUV_LIB NAMES uv PATHS lib/libuv)
+find_path(LIBUV_HEADER_PATH NAMES uv.h PATHS lib/libuv/include uv/include)
+if (NOT (LIBUV_LIB AND LIBUV_HEADER_PATH))
+    MESSAGE(FATAL_ERROR "Failed to find libuv.  Please install libuv.  LIBUV_LIB=${LIBUV_LIB}, LIBUV_HEADER_PATH=${LIBUV_HEADER_PATH}") 
+endif ()
+include_directories(
+    lib/libuv/include
+    uv/include)
+
+# Find protobuf-c
+find_library(PROTOBUFC_LIB NAMES protobuf-c 
+    HINTS /usr/lib64 /usr/lib)
+find_program(PROTOBUFC_EXE NAMES protoc-c)
+if (NOT (PROTOBUFC_LIB AND PROTOBUFC_EXE))
+    MESSAGE(FATAL_ERROR "Failed to find protobuf-c.  Please install protobuf-c.  PROTOBUFC_LIB=${PROTOBUFC_LIB}, PROTOBUFC_EXE=${PROTOBUFC_EXE}")
+endif()
+
+# Find protobuf
+find_library(PROTOC_LIB NAMES libprotoc.so
+    HINTS /usr/lib /usr/lib64)
+find_library(PROTOBUF_LIB NAMES libprotobuf.so
+    HINTS /usr/lib /usr/lib64)
+find_program(PROTOC_EXE NAMES protoc)
+find_path(PROTOC_HEADER_PATH NAMES 
+    google/protobuf/compiler/command_line_interface.h
+    HINTS /usr/include)
+if (NOT (PROTOC_LIB AND PROTOBUF_LIB AND PROTOC_EXE AND PROTOC_HEADER_PATH))
+    MESSAGE(FATAL_ERROR "Failed to find the C++ protobuf libraries, which are needed for RPC code generation.  PROTOC_LIB=${PROTOC_LIB}, PROTOBUF_LIB=${PROTOBUF_LIB}, PROTOC_EXE=${PROTOC_EXE}, PROTOC_HEADER_PATH=${PROTOC_HEADER_PATH}")
+endif ()
+
+include_directories(
+    ${CMAKE_CURRENT_SOURCE_DIR}
+    ${CMAKE_CURRENT_BINARY_DIR}
+    ${PROTOBUF_HEADER_PATH})
+
+include(GenerateProtobufs.cmake NO_POLICY_SCOPE)
+
+set(COMMON_SRCS
+    common/hadoop_err.c
+    common/net.c
+    common/string.c
+    common/test.c
+)
+set(COMMON_DEPS
+    pthread
+)
+
+set(RPC_SRCS
+    rpc/call.c
+    rpc/client_id.c
+    rpc/conn.c
+    rpc/messenger.c
+    rpc/proxy.c
+    rpc/reactor.c
+    rpc/varint.c
+    ${COMMON_PROTOBUF_SRCS}
+)
+set(RPC_DEPS
+    ${LIBUV_LIB}
+    ${PROTOBUFC_LIB}
+)
+
+add_executable(varint-unit rpc/varint-unit.c
+    rpc/varint.c common/test.c)
+add_utest(varint-unit)
+
+add_executable(hadoop_err-unit common/hadoop_err-unit.c
+    common/hadoop_err.c common/test.c)
+add_utest(hadoop_err-unit)
+target_link_libraries(hadoop_err-unit uv)
+
+add_executable(namenode-rpc-unit hdfs/namenode-rpc-unit.c)
+target_link_libraries(namenode-rpc-unit hdfs-core)
+
+add_library(hdfs-core SHARED
+    ${COMMON_SRCS}
+    ${RPC_SRCS}
+    ${HDFS_PROTOBUF_SRCS}
+)
+target_link_libraries(hdfs-core 
+    ${COMMON_DEPS}
+    ${RPC_DEPS}
+)
+set(HDFS_CORE_VERSION_MAJOR 1)
+set(HDFS_CORE_VERSION_MINOR 0)
+set(HDFS_CORE_VERSION_PATCH 0)
+set(HDFS_CORE_VERSION_STRING "${HDFS_CORE_VERSION_MAJOR}.${HDFS_CORE_VERSION_MINOR}.${HDFS_CORE_VERSION_PATCH}")
+set_target_properties(hdfs-core PROPERTIES
+    VERSION ${HDFS_CORE_VERSION_STRING}
+    SOVERSION ${HDFS_CORE_VERSION_MAJOR})
+
+add_library(yarn-core SHARED
+    ${COMMON_SRCS}
+    ${RPC_SRCS}
+    ${YARN_PROTOBUF_SRCS}
+)
+target_link_libraries(yarn-core 
+    ${COMMON_DEPS}
+    ${RPC_DEPS}
+)
+set(YARN_CORE_VERSION_MAJOR 1)
+set(YARN_CORE_VERSION_MINOR 0)
+set(YARN_CORE_VERSION_PATCH 0)
+set(YARN_CORE_VERSION_STRING ${YARN_CORE_VERSION_MAJOR}.${YARN_CORE_VERSION_MINOR}.${YARN_CORE_VERSION_PATCH})
+set_target_properties(yarn-core PROPERTIES
+    VERSION ${YARN_CORE_VERSION_STRING}
+    SOVERSION ${YARN_CORE_VERSION_MAJOR})

+ 93 - 0
hadoop-native-core/GenerateProtobufs.cmake

@@ -0,0 +1,93 @@
+MESSAGE(STATUS "Processing hadoop-core protobuf definitions.")
+
+add_executable(shorten rpc/shorten.c)
+
+include_directories(${PROTOC_HEADER_PATH})
+add_executable(protoc-gen-hrpc rpc/protoc-gen-hrpc.cc)
+target_link_libraries(protoc-gen-hrpc ${PROTOC_LIB} ${PROTOBUF_LIB})
+
+function(DECLARE_PROTOS OUTPUT_SRC_LIST GENERATOR_DIR INCLUDE_DIRS)
+    if (NOT ARGN)
+        message(SEND_ERROR "Error: DECLARE_PROTOS requires protobuf files as arguments.")
+    endif()
+    set(CFILES)
+    set(HFILES)
+    get_filename_component(GENERATOR_DIR "${GENERATOR_DIR}" REALPATH)
+    set(INCLUDE_FLAGS)
+    foreach(IDIR ${INCLUDE_DIRS})
+        set(INCLUDE_FLAGS ${INCLUDE_FLAGS} -I ${IDIR})
+    endforeach()
+    foreach(PB_FILE ${ARGN})
+        get_filename_component(DIRNAME_F ${PB_FILE} PATH)
+        get_filename_component(ABSNAME_F ${PB_FILE} ABSOLUTE)
+        get_filename_component(BASENAME_F ${PB_FILE} NAME_WE)
+        set(PB_C_FILE "${GENERATOR_DIR}/${BASENAME_F}.pb-c.c")
+        set(PB_H_FILE "${GENERATOR_DIR}/${BASENAME_F}.pb-c.h")
+        set(CALL_C_FILE "${GENERATOR_DIR}/${BASENAME_F}.call.c")
+        set(CALL_H_FILE "${GENERATOR_DIR}/${BASENAME_F}.call.h")
+        execute_process(COMMAND ${CMAKE_COMMAND} -E make_directory ${GENERATOR_DIR})
+        add_custom_command(
+            OUTPUT ${PB_C_FILE} ${PB_H_FILE} ${CALL_C_FILE} ${CALL_H_FILE}
+            COMMAND  ${PROTOBUFC_EXE}
+                ARGS --c_out  ${GENERATOR_DIR} ${INCLUDE_FLAGS} --proto_path ${DIRNAME_F} ${ABSNAME_F}
+                COMMENT "Running protoc-c on ${PB_FILE}"
+                DEPENDS ${ABSNAME_F}
+                VERBATIM 
+            COMMAND  "${CMAKE_CURRENT_BINARY_DIR}/shorten"
+                ARGS ${PB_H_FILE}
+                COMMENT "Processing ${PB_H_FILE}"
+                DEPENDS ${ABSNAME_F} shorten
+                VERBATIM 
+            COMMAND "${PROTOC_EXE}"
+                ARGS --plugin=protoc-gen-hrpc --hrpc_out ${GENERATOR_DIR} ${INCLUDE_FLAGS} ${ABSNAME_F}
+                COMMENT "Running HRPC protocol buffer compiler on ${ABSNAME_F}"
+                DEPENDS ${ABSNAME_F} protoc-gen-hrpc
+                VERBATIM
+            )
+        list(APPEND CFILES ${PB_C_FILE} ${CALL_C_FILE})
+        list(APPEND HFILES ${PB_H_FILE} ${PB_H_FILE}.s ${CALL_H_FILE})
+        set_source_files_properties(${PB_C_FILE} ${PB_H_FILE} "${PB_H_FILE}.s" PROPERTIES GENERATED TRUE)
+    endforeach()
+    #MESSAGE(STATUS "OUTPUT_SRC_LIST = ${OUTPUT_SRC_LIST}, CFILES = ${CFILES}, HFILES = ${HFILES}")
+    set(${OUTPUT_SRC_LIST} ${CFILES} ${HFILES} PARENT_SCOPE)
+endfunction()
+
+get_filename_component(R "${CMAKE_CURRENT_LIST_DIR}/.." REALPATH)
+
+# Common protobuf files.  In general, the other subprojects such as HDFS and
+# YARN may rely on definitions in these protobuf files.
+DECLARE_PROTOS(
+    COMMON_PROTOBUF_SRCS
+    ${CMAKE_CURRENT_BINARY_DIR}/protobuf
+    "${R}/hadoop-common-project/hadoop-common/src/main/proto/"
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/IpcConnectionContext.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshCallQueueProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/Security.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ZKFCProtocol.proto
+)
+
+# HDFS protobuf files.
+DECLARE_PROTOS(
+    HDFS_PROTOBUF_SRCS
+    ${CMAKE_CURRENT_BINARY_DIR}/protobuf
+    "${R}/hadoop-common-project/hadoop-common/src/main/proto/;${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/"
+    #${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HAZKInfo.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
+    #${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+)

+ 40 - 0
hadoop-native-core/common/hadoop_err-unit.c

@@ -0,0 +1,40 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "common/test.h"
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <uv.h>
+
+int main(void)
+{
+    struct hadoop_err *err;
+
+    err = hadoop_lerr_alloc(EINVAL, "foo bar baz %d", 101);
+    EXPECT_STR_EQ("org.apache.hadoop.native.HadoopCore.InvalidRequestException: "
+              "foo bar baz 101", hadoop_err_msg(err));
+    EXPECT_INT_EQ(EINVAL, hadoop_err_code(err));
+    hadoop_err_free(err);
+
+    return EXIT_SUCCESS;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 199 - 0
hadoop-native-core/common/hadoop_err.c

@@ -0,0 +1,199 @@
+/**
+ * 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 "hadoop_err.h"
+
+#include <errno.h>
+#include <stdarg.h>
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#include <uv.h>
+
+struct hadoop_err {
+    int code;
+    char *msg;
+    int malloced;
+};
+
+#define SUFFIX ": "
+#define SUFFIX_LEN (sizeof(SUFFIX) - 1)
+
+#define RUNTIME_EXCEPTION_ERROR_CODE EFAULT
+
+#define RUNTIME_EXCEPTION_CLASS \
+    "org.apache.hadoop.native.HadoopCore.RuntimeException"
+
+static const struct hadoop_err HADOOP_OOM_ERR = {
+    RUNTIME_EXCEPTION_ERROR_CODE,
+    RUNTIME_EXCEPTION_CLASS": Failed to allcoate memory for an error message.",
+    0
+};
+
+static const struct hadoop_err HADOOP_VSNPRINTF_ERR = {
+    RUNTIME_EXCEPTION_ERROR_CODE,
+    RUNTIME_EXCEPTION_CLASS": vsnprintf error while preparing an error "
+        "message.",
+    0
+};
+
+static const char *errno_to_class(int code)
+{
+    switch (code) {
+    case RUNTIME_EXCEPTION_ERROR_CODE:
+        return RUNTIME_EXCEPTION_CLASS;
+    case EINVAL:
+        return "org.apache.hadoop.native.HadoopCore.InvalidRequestException";
+    case ENOMEM:
+        return "org.apache.hadoop.native.HadoopCore.OutOfMemory";
+    default:
+        return "org.apache.hadoop.native.HadoopCore.IOException";
+    }
+}
+
+static const char *uv_code_to_class(int code)
+{
+    switch (code) {
+    case UV_EOF:
+        return "org.apache.hadoop.native.HadoopCore.EOFException";
+    case UV_EINVAL:
+        return "org.apache.hadoop.native.HadoopCore.InvalidRequestException";
+    case UV_ECONNREFUSED:
+        return "org.apache.hadoop.native.HadoopCore.ConnectionRefusedException";
+    case UV_ENOMEM:
+        return "org.apache.hadoop.native.HadoopCore.OutOfMemoryException";
+    default:
+        return "org.apache.hadoop.native.HadoopCore.IOException";
+    }
+}
+
+struct hadoop_err *hadoop_lerr_alloc(int code, const char *fmt, ...)
+{
+    struct hadoop_err *err = NULL, *ierr = NULL;
+    va_list ap, ap2;
+    const char *class;
+    char test_buf[1];
+    int fmt_len, class_len, msg_len;
+
+    err = calloc(1, sizeof(*err));
+    if (!err) {
+        ierr = (struct hadoop_err*)&HADOOP_OOM_ERR;
+        goto done;
+    }
+    err->code = code;
+    err->malloced = 1;
+    va_start(ap, fmt);
+    va_copy(ap2, ap);
+    fmt_len = vsnprintf(test_buf, sizeof(test_buf), fmt, ap);
+    if (fmt_len < 0) {
+        ierr = (struct hadoop_err*)&HADOOP_VSNPRINTF_ERR;
+        goto done;
+    }
+    class = errno_to_class(code);
+    class_len = strlen(class);
+    msg_len = class_len + SUFFIX_LEN + fmt_len + 1;
+    err->msg = malloc(msg_len);
+    if (!err->msg) {
+        ierr = (struct hadoop_err*)&HADOOP_OOM_ERR;
+        goto done;
+    }
+    strcpy(err->msg, class);
+    strcpy(err->msg + class_len, SUFFIX);
+    vsprintf(err->msg + class_len + SUFFIX_LEN, fmt, ap2);
+    va_end(ap2);
+    va_end(ap);
+
+done:
+    if (ierr) {
+        hadoop_err_free(err);
+        return ierr;
+    }
+    return err;
+}
+
+struct hadoop_err *hadoop_uverr_alloc(int code, const char *fmt, ...)
+{
+    struct hadoop_err *err = NULL, *ierr = NULL;
+    va_list ap, ap2;
+    const char *class;
+    const char *umsg;
+    char test_buf[1];
+    int fmt_len, umsg_len, class_len, msg_len;
+
+    err = calloc(1, sizeof(*err));
+    if (!err) {
+        ierr = (struct hadoop_err*)&HADOOP_OOM_ERR;
+        goto done;
+    }
+    err->code = code;
+    umsg = uv_strerror(code);
+    umsg_len = strlen(umsg);
+    err->malloced = 1;
+    va_start(ap, fmt);
+    va_copy(ap2, ap);
+    fmt_len = vsnprintf(test_buf, sizeof(test_buf), fmt, ap);
+    if (fmt_len < 0) {
+        ierr = (struct hadoop_err*)&HADOOP_VSNPRINTF_ERR;
+        goto done;
+    }
+    class = uv_code_to_class(err->code);
+    class_len = strlen(class);
+    msg_len = class_len + SUFFIX_LEN + umsg_len + SUFFIX_LEN + fmt_len + 1;
+    err->msg = malloc(msg_len);
+    if (!err->msg) {
+        ierr = (struct hadoop_err*)&HADOOP_OOM_ERR;
+        goto done;
+    }
+    strcpy(err->msg, class);
+    strcpy(err->msg + class_len, SUFFIX);
+    strcpy(err->msg + class_len + SUFFIX_LEN, umsg);
+    strcpy(err->msg + class_len + SUFFIX_LEN + umsg_len, SUFFIX);
+    vsprintf(err->msg + class_len + SUFFIX_LEN + umsg_len + SUFFIX_LEN,
+        fmt, ap2);
+    va_end(ap2);
+    va_end(ap);
+
+done:
+    if (ierr) {
+        hadoop_err_free(err);
+        return ierr;
+    }
+    return err;
+}
+
+void hadoop_err_free(struct hadoop_err *err)
+{
+    if (!err)
+        return;
+    if (!err->malloced)
+        return;
+    free(err->msg);
+    free(err);
+}
+
+int hadoop_err_code(const struct hadoop_err *err)
+{
+    return err->code;
+}
+
+const char *hadoop_err_msg(const struct hadoop_err *err)
+{
+    return err->msg;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 81 - 0
hadoop-native-core/common/hadoop_err.h

@@ -0,0 +1,81 @@
+/**
+ * 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 HADOOP_CORE_COMMON_HADOOP_ERR
+#define HADOOP_CORE_COMMON_HADOOP_ERR
+
+#include <uv.h> /* for uv_loop_t */
+
+/**
+ * A Hadoop error.  This is the libhadoop-core version of an IOException.
+ */
+struct hadoop_err;
+
+/**
+ * Allocate a new local error.
+ *
+ * @param code          Error code.
+ * @param fmt           printf-style format.
+ * @param ...           printf-style arguments.
+ *
+ * @return              A new error message.  This will never be NULL.
+ */
+struct hadoop_err *hadoop_lerr_alloc(int code, const char *fmt, ...)
+    __attribute__((format(printf, 2, 3)));
+
+/**
+ * Allocate a new error object based on a libuv error.
+ *
+ * @param loop          The libuv loop to check.
+ * @param fmt           printf-style format.
+ * @param ...           printf-style arguments.
+ *
+ * @return              A new error message.  This will never be NULL.
+ */
+struct hadoop_err *hadoop_uverr_alloc(int cod, const char *fmt, ...)
+    __attribute__((format(printf, 2, 3)));
+
+/**
+ * Given a hadoop error, get the error code.
+ *
+ * @param err       The hadoop error.
+ *
+ * @return          The error code.
+ */
+int hadoop_err_code(const struct hadoop_err *err);
+
+/**
+ * Given a hadoop error, get the error message.
+ *
+ * @param err       The hadoop error.
+ *
+ * @return          The error message.  Valid until the hadoop_err
+ *                  object is freed.
+ */
+const char *hadoop_err_msg(const struct hadoop_err *err);
+
+/**
+ * Free a hadoop error.
+ *
+ * @param err       The hadoop error.
+ */
+void hadoop_err_free(struct hadoop_err *err);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 31 - 0
hadoop-native-core/common/net.c

@@ -0,0 +1,31 @@
+/**
+ * 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 <netinet/in.h>
+#include <unistd.h>
+#include <uv.h>
+
+const char *net_ipv4_name(struct sockaddr_in *src, char *dst, size_t size)
+{
+  if (uv_ip4_name(src, dst, size) < 0) {
+    return "(uv_ip4_name error)";
+  }
+  return dst;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 30 - 0
hadoop-native-core/common/net.h

@@ -0,0 +1,30 @@
+/**
+ * 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 HADOOP_CORE_COMMON_NET
+#define HADOOP_CORE_COMMON_NET
+
+#include <stddef.h>
+
+struct sockaddr_in;
+
+const char *net_ipv4_name(struct sockaddr_in *src, char *dst, size_t size);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 694 - 0
hadoop-native-core/common/queue.h

@@ -0,0 +1,694 @@
+/*-
+ * Copyright (c) 1991, 1993
+ *	The Regents of the University of California.  All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 4. Neither the name of the University nor the names of its contributors
+ *    may be used to endorse or promote products derived from this software
+ *    without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND
+ * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED.  IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE
+ * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+ * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+ * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+ * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ *
+ *	@(#)queue.h	8.5 (Berkeley) 8/20/94
+ * $FreeBSD$
+ */
+
+#ifndef _SYS_QUEUE_H_
+#define	_SYS_QUEUE_H_
+
+#include <sys/cdefs.h>
+
+/*
+ * This file defines four types of data structures: singly-linked lists,
+ * singly-linked tail queues, lists and tail queues.
+ *
+ * A singly-linked list is headed by a single forward pointer. The elements
+ * are singly linked for minimum space and pointer manipulation overhead at
+ * the expense of O(n) removal for arbitrary elements. New elements can be
+ * added to the list after an existing element or at the head of the list.
+ * Elements being removed from the head of the list should use the explicit
+ * macro for this purpose for optimum efficiency. A singly-linked list may
+ * only be traversed in the forward direction.  Singly-linked lists are ideal
+ * for applications with large datasets and few or no removals or for
+ * implementing a LIFO queue.
+ *
+ * A singly-linked tail queue is headed by a pair of pointers, one to the
+ * head of the list and the other to the tail of the list. The elements are
+ * singly linked for minimum space and pointer manipulation overhead at the
+ * expense of O(n) removal for arbitrary elements. New elements can be added
+ * to the list after an existing element, at the head of the list, or at the
+ * end of the list. Elements being removed from the head of the tail queue
+ * should use the explicit macro for this purpose for optimum efficiency.
+ * A singly-linked tail queue may only be traversed in the forward direction.
+ * Singly-linked tail queues are ideal for applications with large datasets
+ * and few or no removals or for implementing a FIFO queue.
+ *
+ * A list is headed by a single forward pointer (or an array of forward
+ * pointers for a hash table header). The elements are doubly linked
+ * so that an arbitrary element can be removed without a need to
+ * traverse the list. New elements can be added to the list before
+ * or after an existing element or at the head of the list. A list
+ * may be traversed in either direction.
+ *
+ * A tail queue is headed by a pair of pointers, one to the head of the
+ * list and the other to the tail of the list. The elements are doubly
+ * linked so that an arbitrary element can be removed without a need to
+ * traverse the list. New elements can be added to the list before or
+ * after an existing element, at the head of the list, or at the end of
+ * the list. A tail queue may be traversed in either direction.
+ *
+ * For details on the use of these macros, see the queue(3) manual page.
+ *
+ *
+ *				SLIST	LIST	STAILQ	TAILQ
+ * _HEAD			+	+	+	+
+ * _HEAD_INITIALIZER		+	+	+	+
+ * _ENTRY			+	+	+	+
+ * _INIT			+	+	+	+
+ * _EMPTY			+	+	+	+
+ * _FIRST			+	+	+	+
+ * _NEXT			+	+	+	+
+ * _PREV			-	+	-	+
+ * _LAST			-	-	+	+
+ * _FOREACH			+	+	+	+
+ * _FOREACH_FROM		+	+	+	+
+ * _FOREACH_SAFE		+	+	+	+
+ * _FOREACH_FROM_SAFE		+	+	+	+
+ * _FOREACH_REVERSE		-	-	-	+
+ * _FOREACH_REVERSE_FROM	-	-	-	+
+ * _FOREACH_REVERSE_SAFE	-	-	-	+
+ * _FOREACH_REVERSE_FROM_SAFE	-	-	-	+
+ * _INSERT_HEAD			+	+	+	+
+ * _INSERT_BEFORE		-	+	-	+
+ * _INSERT_AFTER		+	+	+	+
+ * _INSERT_TAIL			-	-	+	+
+ * _CONCAT			-	-	+	+
+ * _REMOVE_AFTER		+	-	+	-
+ * _REMOVE_HEAD			+	-	+	-
+ * _REMOVE			+	+	+	+
+ * _SWAP			+	+	+	+
+ *
+ */
+#ifdef QUEUE_MACRO_DEBUG
+/* Store the last 2 places the queue element or head was altered */
+struct qm_trace {
+	unsigned long	 lastline;
+	unsigned long	 prevline;
+	const char	*lastfile;
+	const char	*prevfile;
+};
+
+#define	TRACEBUF	struct qm_trace trace;
+#define	TRACEBUF_INITIALIZER	{ __FILE__, __LINE__, NULL, 0 } ,
+#define	TRASHIT(x)	do {(x) = (void *)-1;} while (0)
+#define	QMD_SAVELINK(name, link)	void **name = (void *)&(link)
+
+#define	QMD_TRACE_HEAD(head) do {					\
+	(head)->trace.prevline = (head)->trace.lastline;		\
+	(head)->trace.prevfile = (head)->trace.lastfile;		\
+	(head)->trace.lastline = __LINE__;				\
+	(head)->trace.lastfile = __FILE__;				\
+} while (0)
+
+#define	QMD_TRACE_ELEM(elem) do {					\
+	(elem)->trace.prevline = (elem)->trace.lastline;		\
+	(elem)->trace.prevfile = (elem)->trace.lastfile;		\
+	(elem)->trace.lastline = __LINE__;				\
+	(elem)->trace.lastfile = __FILE__;				\
+} while (0)
+
+#else
+#define	QMD_TRACE_ELEM(elem)
+#define	QMD_TRACE_HEAD(head)
+#define	QMD_SAVELINK(name, link)
+#define	TRACEBUF
+#define	TRACEBUF_INITIALIZER
+#define	TRASHIT(x)
+#endif	/* QUEUE_MACRO_DEBUG */
+
+/*
+ * Singly-linked List declarations.
+ */
+#define	SLIST_HEAD(name, type)						\
+struct name {								\
+	struct type *slh_first;	/* first element */			\
+}
+
+#define	SLIST_HEAD_INITIALIZER(head)					\
+	{ NULL }
+
+#define	SLIST_ENTRY(type)						\
+struct {								\
+	struct type *sle_next;	/* next element */			\
+}
+
+/*
+ * Singly-linked List functions.
+ */
+#define	SLIST_EMPTY(head)	((head)->slh_first == NULL)
+
+#define	SLIST_FIRST(head)	((head)->slh_first)
+
+#define	SLIST_FOREACH(var, head, field)					\
+	for ((var) = SLIST_FIRST((head));				\
+	    (var);							\
+	    (var) = SLIST_NEXT((var), field))
+
+#define	SLIST_FOREACH_FROM(var, head, field)				\
+	for ((var) = ((var) ? (var) : SLIST_FIRST((head)));		\
+	    (var);							\
+	    (var) = SLIST_NEXT((var), field))
+
+#define	SLIST_FOREACH_SAFE(var, head, field, tvar)			\
+	for ((var) = SLIST_FIRST((head));				\
+	    (var) && ((tvar) = SLIST_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	SLIST_FOREACH_FROM_SAFE(var, head, field, tvar)			\
+	for ((var) = ((var) ? (var) : SLIST_FIRST((head)));		\
+	    (var) && ((tvar) = SLIST_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	SLIST_FOREACH_PREVPTR(var, varp, head, field)			\
+	for ((varp) = &SLIST_FIRST((head));				\
+	    ((var) = *(varp)) != NULL;					\
+	    (varp) = &SLIST_NEXT((var), field))
+
+#define	SLIST_INIT(head) do {						\
+	SLIST_FIRST((head)) = NULL;					\
+} while (0)
+
+#define	SLIST_INSERT_AFTER(slistelm, elm, field) do {			\
+	SLIST_NEXT((elm), field) = SLIST_NEXT((slistelm), field);	\
+	SLIST_NEXT((slistelm), field) = (elm);				\
+} while (0)
+
+#define	SLIST_INSERT_HEAD(head, elm, field) do {			\
+	SLIST_NEXT((elm), field) = SLIST_FIRST((head));			\
+	SLIST_FIRST((head)) = (elm);					\
+} while (0)
+
+#define	SLIST_NEXT(elm, field)	((elm)->field.sle_next)
+
+#define	SLIST_REMOVE(head, elm, type, field) do {			\
+	QMD_SAVELINK(oldnext, (elm)->field.sle_next);			\
+	if (SLIST_FIRST((head)) == (elm)) {				\
+		SLIST_REMOVE_HEAD((head), field);			\
+	}								\
+	else {								\
+		struct type *curelm = SLIST_FIRST((head));		\
+		while (SLIST_NEXT(curelm, field) != (elm))		\
+			curelm = SLIST_NEXT(curelm, field);		\
+		SLIST_REMOVE_AFTER(curelm, field);			\
+	}								\
+	TRASHIT(*oldnext);						\
+} while (0)
+
+#define SLIST_REMOVE_AFTER(elm, field) do {				\
+	SLIST_NEXT(elm, field) =					\
+	    SLIST_NEXT(SLIST_NEXT(elm, field), field);			\
+} while (0)
+
+#define	SLIST_REMOVE_HEAD(head, field) do {				\
+	SLIST_FIRST((head)) = SLIST_NEXT(SLIST_FIRST((head)), field);	\
+} while (0)
+
+#define SLIST_SWAP(head1, head2, type) do {				\
+	struct type *swap_first = SLIST_FIRST(head1);			\
+	SLIST_FIRST(head1) = SLIST_FIRST(head2);			\
+	SLIST_FIRST(head2) = swap_first;				\
+} while (0)
+
+/*
+ * Singly-linked Tail queue declarations.
+ */
+#define	STAILQ_HEAD(name, type)						\
+struct name {								\
+	struct type *stqh_first;/* first element */			\
+	struct type **stqh_last;/* addr of last next element */		\
+}
+
+#define	STAILQ_HEAD_INITIALIZER(head)					\
+	{ NULL, &(head).stqh_first }
+
+#define	STAILQ_ENTRY(type)						\
+struct {								\
+	struct type *stqe_next;	/* next element */			\
+}
+
+/*
+ * Singly-linked Tail queue functions.
+ */
+#define	STAILQ_CONCAT(head1, head2) do {				\
+	if (!STAILQ_EMPTY((head2))) {					\
+		*(head1)->stqh_last = (head2)->stqh_first;		\
+		(head1)->stqh_last = (head2)->stqh_last;		\
+		STAILQ_INIT((head2));					\
+	}								\
+} while (0)
+
+#define	STAILQ_EMPTY(head)	((head)->stqh_first == NULL)
+
+#define	STAILQ_FIRST(head)	((head)->stqh_first)
+
+#define	STAILQ_FOREACH(var, head, field)				\
+	for((var) = STAILQ_FIRST((head));				\
+	   (var);							\
+	   (var) = STAILQ_NEXT((var), field))
+
+#define	STAILQ_FOREACH_FROM(var, head, field)				\
+	for ((var) = ((var) ? (var) : STAILQ_FIRST((head)));		\
+	   (var);							\
+	   (var) = STAILQ_NEXT((var), field))
+
+#define	STAILQ_FOREACH_SAFE(var, head, field, tvar)			\
+	for ((var) = STAILQ_FIRST((head));				\
+	    (var) && ((tvar) = STAILQ_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	STAILQ_FOREACH_FROM_SAFE(var, head, field, tvar)		\
+	for ((var) = ((var) ? (var) : STAILQ_FIRST((head)));		\
+	    (var) && ((tvar) = STAILQ_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	STAILQ_INIT(head) do {						\
+	STAILQ_FIRST((head)) = NULL;					\
+	(head)->stqh_last = &STAILQ_FIRST((head));			\
+} while (0)
+
+#define	STAILQ_INSERT_AFTER(head, tqelm, elm, field) do {		\
+	if ((STAILQ_NEXT((elm), field) = STAILQ_NEXT((tqelm), field)) == NULL)\
+		(head)->stqh_last = &STAILQ_NEXT((elm), field);		\
+	STAILQ_NEXT((tqelm), field) = (elm);				\
+} while (0)
+
+#define	STAILQ_INSERT_HEAD(head, elm, field) do {			\
+	if ((STAILQ_NEXT((elm), field) = STAILQ_FIRST((head))) == NULL)	\
+		(head)->stqh_last = &STAILQ_NEXT((elm), field);		\
+	STAILQ_FIRST((head)) = (elm);					\
+} while (0)
+
+#define	STAILQ_INSERT_TAIL(head, elm, field) do {			\
+	STAILQ_NEXT((elm), field) = NULL;				\
+	*(head)->stqh_last = (elm);					\
+	(head)->stqh_last = &STAILQ_NEXT((elm), field);			\
+} while (0)
+
+#define	STAILQ_LAST(head, type, field)					\
+	(STAILQ_EMPTY((head)) ? NULL :					\
+	    __containerof((head)->stqh_last, struct type, field.stqe_next))
+
+#define	STAILQ_NEXT(elm, field)	((elm)->field.stqe_next)
+
+#define	STAILQ_REMOVE(head, elm, type, field) do {			\
+	QMD_SAVELINK(oldnext, (elm)->field.stqe_next);			\
+	if (STAILQ_FIRST((head)) == (elm)) {				\
+		STAILQ_REMOVE_HEAD((head), field);			\
+	}								\
+	else {								\
+		struct type *curelm = STAILQ_FIRST((head));		\
+		while (STAILQ_NEXT(curelm, field) != (elm))		\
+			curelm = STAILQ_NEXT(curelm, field);		\
+		STAILQ_REMOVE_AFTER(head, curelm, field);		\
+	}								\
+	TRASHIT(*oldnext);						\
+} while (0)
+
+#define STAILQ_REMOVE_AFTER(head, elm, field) do {			\
+	if ((STAILQ_NEXT(elm, field) =					\
+	     STAILQ_NEXT(STAILQ_NEXT(elm, field), field)) == NULL)	\
+		(head)->stqh_last = &STAILQ_NEXT((elm), field);		\
+} while (0)
+
+#define	STAILQ_REMOVE_HEAD(head, field) do {				\
+	if ((STAILQ_FIRST((head)) =					\
+	     STAILQ_NEXT(STAILQ_FIRST((head)), field)) == NULL)		\
+		(head)->stqh_last = &STAILQ_FIRST((head));		\
+} while (0)
+
+#define STAILQ_SWAP(head1, head2, type) do {				\
+	struct type *swap_first = STAILQ_FIRST(head1);			\
+	struct type **swap_last = (head1)->stqh_last;			\
+	STAILQ_FIRST(head1) = STAILQ_FIRST(head2);			\
+	(head1)->stqh_last = (head2)->stqh_last;			\
+	STAILQ_FIRST(head2) = swap_first;				\
+	(head2)->stqh_last = swap_last;					\
+	if (STAILQ_EMPTY(head1))					\
+		(head1)->stqh_last = &STAILQ_FIRST(head1);		\
+	if (STAILQ_EMPTY(head2))					\
+		(head2)->stqh_last = &STAILQ_FIRST(head2);		\
+} while (0)
+
+
+/*
+ * List declarations.
+ */
+#define	LIST_HEAD(name, type)						\
+struct name {								\
+	struct type *lh_first;	/* first element */			\
+}
+
+#define	LIST_HEAD_INITIALIZER(head)					\
+	{ NULL }
+
+#define	LIST_ENTRY(type)						\
+struct {								\
+	struct type *le_next;	/* next element */			\
+	struct type **le_prev;	/* address of previous next element */	\
+}
+
+/*
+ * List functions.
+ */
+
+#if (defined(_KERNEL) && defined(INVARIANTS))
+#define	QMD_LIST_CHECK_HEAD(head, field) do {				\
+	if (LIST_FIRST((head)) != NULL &&				\
+	    LIST_FIRST((head))->field.le_prev !=			\
+	     &LIST_FIRST((head)))					\
+		panic("Bad list head %p first->prev != head", (head));	\
+} while (0)
+
+#define	QMD_LIST_CHECK_NEXT(elm, field) do {				\
+	if (LIST_NEXT((elm), field) != NULL &&				\
+	    LIST_NEXT((elm), field)->field.le_prev !=			\
+	     &((elm)->field.le_next))					\
+	     	panic("Bad link elm %p next->prev != elm", (elm));	\
+} while (0)
+
+#define	QMD_LIST_CHECK_PREV(elm, field) do {				\
+	if (*(elm)->field.le_prev != (elm))				\
+		panic("Bad link elm %p prev->next != elm", (elm));	\
+} while (0)
+#else
+#define	QMD_LIST_CHECK_HEAD(head, field)
+#define	QMD_LIST_CHECK_NEXT(elm, field)
+#define	QMD_LIST_CHECK_PREV(elm, field)
+#endif /* (_KERNEL && INVARIANTS) */
+
+#define	LIST_EMPTY(head)	((head)->lh_first == NULL)
+
+#define	LIST_FIRST(head)	((head)->lh_first)
+
+#define	LIST_FOREACH(var, head, field)					\
+	for ((var) = LIST_FIRST((head));				\
+	    (var);							\
+	    (var) = LIST_NEXT((var), field))
+
+#define	LIST_FOREACH_FROM(var, head, field)				\
+	for ((var) = ((var) ? (var) : LIST_FIRST((head)));		\
+	    (var);							\
+	    (var) = LIST_NEXT((var), field))
+
+#define	LIST_FOREACH_SAFE(var, head, field, tvar)			\
+	for ((var) = LIST_FIRST((head));				\
+	    (var) && ((tvar) = LIST_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	LIST_FOREACH_FROM_SAFE(var, head, field, tvar)			\
+	for ((var) = ((var) ? (var) : LIST_FIRST((head)));		\
+	    (var) && ((tvar) = LIST_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	LIST_INIT(head) do {						\
+	LIST_FIRST((head)) = NULL;					\
+} while (0)
+
+#define	LIST_INSERT_AFTER(listelm, elm, field) do {			\
+	QMD_LIST_CHECK_NEXT(listelm, field);				\
+	if ((LIST_NEXT((elm), field) = LIST_NEXT((listelm), field)) != NULL)\
+		LIST_NEXT((listelm), field)->field.le_prev =		\
+		    &LIST_NEXT((elm), field);				\
+	LIST_NEXT((listelm), field) = (elm);				\
+	(elm)->field.le_prev = &LIST_NEXT((listelm), field);		\
+} while (0)
+
+#define	LIST_INSERT_BEFORE(listelm, elm, field) do {			\
+	QMD_LIST_CHECK_PREV(listelm, field);				\
+	(elm)->field.le_prev = (listelm)->field.le_prev;		\
+	LIST_NEXT((elm), field) = (listelm);				\
+	*(listelm)->field.le_prev = (elm);				\
+	(listelm)->field.le_prev = &LIST_NEXT((elm), field);		\
+} while (0)
+
+#define	LIST_INSERT_HEAD(head, elm, field) do {				\
+	QMD_LIST_CHECK_HEAD((head), field);				\
+	if ((LIST_NEXT((elm), field) = LIST_FIRST((head))) != NULL)	\
+		LIST_FIRST((head))->field.le_prev = &LIST_NEXT((elm), field);\
+	LIST_FIRST((head)) = (elm);					\
+	(elm)->field.le_prev = &LIST_FIRST((head));			\
+} while (0)
+
+#define	LIST_NEXT(elm, field)	((elm)->field.le_next)
+
+#define	LIST_PREV(elm, head, type, field)				\
+	((elm)->field.le_prev == &LIST_FIRST((head)) ? NULL :		\
+	    __containerof((elm)->field.le_prev, struct type, field.le_next))
+
+#define	LIST_REMOVE(elm, field) do {					\
+	QMD_SAVELINK(oldnext, (elm)->field.le_next);			\
+	QMD_SAVELINK(oldprev, (elm)->field.le_prev);			\
+	QMD_LIST_CHECK_NEXT(elm, field);				\
+	QMD_LIST_CHECK_PREV(elm, field);				\
+	if (LIST_NEXT((elm), field) != NULL)				\
+		LIST_NEXT((elm), field)->field.le_prev = 		\
+		    (elm)->field.le_prev;				\
+	*(elm)->field.le_prev = LIST_NEXT((elm), field);		\
+	TRASHIT(*oldnext);						\
+	TRASHIT(*oldprev);						\
+} while (0)
+
+#define LIST_SWAP(head1, head2, type, field) do {			\
+	struct type *swap_tmp = LIST_FIRST((head1));			\
+	LIST_FIRST((head1)) = LIST_FIRST((head2));			\
+	LIST_FIRST((head2)) = swap_tmp;					\
+	if ((swap_tmp = LIST_FIRST((head1))) != NULL)			\
+		swap_tmp->field.le_prev = &LIST_FIRST((head1));		\
+	if ((swap_tmp = LIST_FIRST((head2))) != NULL)			\
+		swap_tmp->field.le_prev = &LIST_FIRST((head2));		\
+} while (0)
+
+/*
+ * Tail queue declarations.
+ */
+#define	TAILQ_HEAD(name, type)						\
+struct name {								\
+	struct type *tqh_first;	/* first element */			\
+	struct type **tqh_last;	/* addr of last next element */		\
+	TRACEBUF							\
+}
+
+#define	TAILQ_HEAD_INITIALIZER(head)					\
+	{ NULL, &(head).tqh_first, TRACEBUF_INITIALIZER }
+
+#define	TAILQ_ENTRY(type)						\
+struct {								\
+	struct type *tqe_next;	/* next element */			\
+	struct type **tqe_prev;	/* address of previous next element */	\
+	TRACEBUF							\
+}
+
+/*
+ * Tail queue functions.
+ */
+#if (defined(_KERNEL) && defined(INVARIANTS))
+#define	QMD_TAILQ_CHECK_HEAD(head, field) do {				\
+	if (!TAILQ_EMPTY(head) &&					\
+	    TAILQ_FIRST((head))->field.tqe_prev !=			\
+	     &TAILQ_FIRST((head)))					\
+		panic("Bad tailq head %p first->prev != head", (head));	\
+} while (0)
+
+#define	QMD_TAILQ_CHECK_TAIL(head, field) do {				\
+	if (*(head)->tqh_last != NULL)					\
+	    	panic("Bad tailq NEXT(%p->tqh_last) != NULL", (head)); 	\
+} while (0)
+
+#define	QMD_TAILQ_CHECK_NEXT(elm, field) do {				\
+	if (TAILQ_NEXT((elm), field) != NULL &&				\
+	    TAILQ_NEXT((elm), field)->field.tqe_prev !=			\
+	     &((elm)->field.tqe_next))					\
+		panic("Bad link elm %p next->prev != elm", (elm));	\
+} while (0)
+
+#define	QMD_TAILQ_CHECK_PREV(elm, field) do {				\
+	if (*(elm)->field.tqe_prev != (elm))				\
+		panic("Bad link elm %p prev->next != elm", (elm));	\
+} while (0)
+#else
+#define	QMD_TAILQ_CHECK_HEAD(head, field)
+#define	QMD_TAILQ_CHECK_TAIL(head, headname)
+#define	QMD_TAILQ_CHECK_NEXT(elm, field)
+#define	QMD_TAILQ_CHECK_PREV(elm, field)
+#endif /* (_KERNEL && INVARIANTS) */
+
+#define	TAILQ_CONCAT(head1, head2, field) do {				\
+	if (!TAILQ_EMPTY(head2)) {					\
+		*(head1)->tqh_last = (head2)->tqh_first;		\
+		(head2)->tqh_first->field.tqe_prev = (head1)->tqh_last;	\
+		(head1)->tqh_last = (head2)->tqh_last;			\
+		TAILQ_INIT((head2));					\
+		QMD_TRACE_HEAD(head1);					\
+		QMD_TRACE_HEAD(head2);					\
+	}								\
+} while (0)
+
+#define	TAILQ_EMPTY(head)	((head)->tqh_first == NULL)
+
+#define	TAILQ_FIRST(head)	((head)->tqh_first)
+
+#define	TAILQ_FOREACH(var, head, field)					\
+	for ((var) = TAILQ_FIRST((head));				\
+	    (var);							\
+	    (var) = TAILQ_NEXT((var), field))
+
+#define	TAILQ_FOREACH_FROM(var, head, field)				\
+	for ((var) = ((var) ? (var) : TAILQ_FIRST((head)));		\
+	    (var);							\
+	    (var) = TAILQ_NEXT((var), field))
+
+#define	TAILQ_FOREACH_SAFE(var, head, field, tvar)			\
+	for ((var) = TAILQ_FIRST((head));				\
+	    (var) && ((tvar) = TAILQ_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	TAILQ_FOREACH_FROM_SAFE(var, head, field, tvar)			\
+	for ((var) = ((var) ? (var) : TAILQ_FIRST((head)));		\
+	    (var) && ((tvar) = TAILQ_NEXT((var), field), 1);		\
+	    (var) = (tvar))
+
+#define	TAILQ_FOREACH_REVERSE(var, head, headname, field)		\
+	for ((var) = TAILQ_LAST((head), headname);			\
+	    (var);							\
+	    (var) = TAILQ_PREV((var), headname, field))
+
+#define	TAILQ_FOREACH_REVERSE_FROM(var, head, headname, field)		\
+	for ((var) = ((var) ? (var) : TAILQ_LAST((head), headname));	\
+	    (var);							\
+	    (var) = TAILQ_PREV((var), headname, field))
+
+#define	TAILQ_FOREACH_REVERSE_SAFE(var, head, headname, field, tvar)	\
+	for ((var) = TAILQ_LAST((head), headname);			\
+	    (var) && ((tvar) = TAILQ_PREV((var), headname, field), 1);	\
+	    (var) = (tvar))
+
+#define	TAILQ_FOREACH_REVERSE_FROM_SAFE(var, head, headname, field, tvar) \
+	for ((var) = ((var) ? (var) : TAILQ_LAST((head), headname));	\
+	    (var) && ((tvar) = TAILQ_PREV((var), headname, field), 1);	\
+	    (var) = (tvar))
+
+#define	TAILQ_INIT(head) do {						\
+	TAILQ_FIRST((head)) = NULL;					\
+	(head)->tqh_last = &TAILQ_FIRST((head));			\
+	QMD_TRACE_HEAD(head);						\
+} while (0)
+
+#define	TAILQ_INSERT_AFTER(head, listelm, elm, field) do {		\
+	QMD_TAILQ_CHECK_NEXT(listelm, field);				\
+	if ((TAILQ_NEXT((elm), field) = TAILQ_NEXT((listelm), field)) != NULL)\
+		TAILQ_NEXT((elm), field)->field.tqe_prev = 		\
+		    &TAILQ_NEXT((elm), field);				\
+	else {								\
+		(head)->tqh_last = &TAILQ_NEXT((elm), field);		\
+		QMD_TRACE_HEAD(head);					\
+	}								\
+	TAILQ_NEXT((listelm), field) = (elm);				\
+	(elm)->field.tqe_prev = &TAILQ_NEXT((listelm), field);		\
+	QMD_TRACE_ELEM(&(elm)->field);					\
+	QMD_TRACE_ELEM(&listelm->field);				\
+} while (0)
+
+#define	TAILQ_INSERT_BEFORE(listelm, elm, field) do {			\
+	QMD_TAILQ_CHECK_PREV(listelm, field);				\
+	(elm)->field.tqe_prev = (listelm)->field.tqe_prev;		\
+	TAILQ_NEXT((elm), field) = (listelm);				\
+	*(listelm)->field.tqe_prev = (elm);				\
+	(listelm)->field.tqe_prev = &TAILQ_NEXT((elm), field);		\
+	QMD_TRACE_ELEM(&(elm)->field);					\
+	QMD_TRACE_ELEM(&listelm->field);				\
+} while (0)
+
+#define	TAILQ_INSERT_HEAD(head, elm, field) do {			\
+	QMD_TAILQ_CHECK_HEAD(head, field);				\
+	if ((TAILQ_NEXT((elm), field) = TAILQ_FIRST((head))) != NULL)	\
+		TAILQ_FIRST((head))->field.tqe_prev =			\
+		    &TAILQ_NEXT((elm), field);				\
+	else								\
+		(head)->tqh_last = &TAILQ_NEXT((elm), field);		\
+	TAILQ_FIRST((head)) = (elm);					\
+	(elm)->field.tqe_prev = &TAILQ_FIRST((head));			\
+	QMD_TRACE_HEAD(head);						\
+	QMD_TRACE_ELEM(&(elm)->field);					\
+} while (0)
+
+#define	TAILQ_INSERT_TAIL(head, elm, field) do {			\
+	QMD_TAILQ_CHECK_TAIL(head, field);				\
+	TAILQ_NEXT((elm), field) = NULL;				\
+	(elm)->field.tqe_prev = (head)->tqh_last;			\
+	*(head)->tqh_last = (elm);					\
+	(head)->tqh_last = &TAILQ_NEXT((elm), field);			\
+	QMD_TRACE_HEAD(head);						\
+	QMD_TRACE_ELEM(&(elm)->field);					\
+} while (0)
+
+#define	TAILQ_LAST(head, headname)					\
+	(*(((struct headname *)((head)->tqh_last))->tqh_last))
+
+#define	TAILQ_NEXT(elm, field) ((elm)->field.tqe_next)
+
+#define	TAILQ_PREV(elm, headname, field)				\
+	(*(((struct headname *)((elm)->field.tqe_prev))->tqh_last))
+
+#define	TAILQ_REMOVE(head, elm, field) do {				\
+	QMD_SAVELINK(oldnext, (elm)->field.tqe_next);			\
+	QMD_SAVELINK(oldprev, (elm)->field.tqe_prev);			\
+	QMD_TAILQ_CHECK_NEXT(elm, field);				\
+	QMD_TAILQ_CHECK_PREV(elm, field);				\
+	if ((TAILQ_NEXT((elm), field)) != NULL)				\
+		TAILQ_NEXT((elm), field)->field.tqe_prev = 		\
+		    (elm)->field.tqe_prev;				\
+	else {								\
+		(head)->tqh_last = (elm)->field.tqe_prev;		\
+		QMD_TRACE_HEAD(head);					\
+	}								\
+	*(elm)->field.tqe_prev = TAILQ_NEXT((elm), field);		\
+	TRASHIT(*oldnext);						\
+	TRASHIT(*oldprev);						\
+	QMD_TRACE_ELEM(&(elm)->field);					\
+} while (0)
+
+#define TAILQ_SWAP(head1, head2, type, field) do {			\
+	struct type *swap_first = (head1)->tqh_first;			\
+	struct type **swap_last = (head1)->tqh_last;			\
+	(head1)->tqh_first = (head2)->tqh_first;			\
+	(head1)->tqh_last = (head2)->tqh_last;				\
+	(head2)->tqh_first = swap_first;				\
+	(head2)->tqh_last = swap_last;					\
+	if ((swap_first = (head1)->tqh_first) != NULL)			\
+		swap_first->field.tqe_prev = &(head1)->tqh_first;	\
+	else								\
+		(head1)->tqh_last = &(head1)->tqh_first;		\
+	if ((swap_first = (head2)->tqh_first) != NULL)			\
+		swap_first->field.tqe_prev = &(head2)->tqh_first;	\
+	else								\
+		(head2)->tqh_last = &(head2)->tqh_first;		\
+} while (0)
+
+#endif /* !_SYS_QUEUE_H_ */

+ 46 - 0
hadoop-native-core/common/string.c

@@ -0,0 +1,46 @@
+/**
+ * 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 "common/string.h"
+
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+void hex_buf_print(FILE *fp, const void *buf, int32_t buf_len,
+                   const char *fmt, ...)
+{
+    int32_t i, j = 0;
+    va_list ap;
+    const uint8_t *b = buf;
+
+    va_start(ap, fmt);
+    vfprintf(fp, fmt, ap);
+    va_end(ap);
+    for (i = 0; i < buf_len; i++) {
+        const char *suffix = " ";
+
+        if (++j == 8) {
+            suffix = "\n";
+            j = 0;
+        }
+        fprintf(fp, "%02x%s", b[i], suffix);
+    }
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 39 - 0
hadoop-native-core/common/string.h

@@ -0,0 +1,39 @@
+/**
+ * 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 HADOOP_CORE_COMMON_STRING
+#define HADOOP_CORE_COMMON_STRING
+
+#include <stdio.h>
+#include <stdint.h>
+
+/**
+ * Print out a buffer in hexadecimal.
+ *
+ * @param fp        The FILE* to print to.
+ * @param buf       The buffer to print.
+ * @param buf_len   The length of the buffer.
+ * @param fmt       Printf-style format.
+ * @param ...       Printf-style arguments.
+ */
+void hex_buf_print(FILE *fp, const void *buf, int32_t buf_len,
+        const char *fmt, ...) __attribute__((format(printf, 4, 5)));
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 136 - 0
hadoop-native-core/common/test.c

@@ -0,0 +1,136 @@
+/**
+ * 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 "common/test.h"
+
+#include <dirent.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <stdarg.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+
+void fail(const char *fmt, ...)
+{
+    va_list ap;
+
+    va_start(ap, fmt);
+    vfprintf(stderr, fmt, ap);
+    va_end(ap);
+    exit(1);
+}
+
+static int vexpect_decode(const char *expected, const char *text, int ty,
+        const char *str)
+{
+    switch (ty) {
+    case TEST_ERROR_EQ:
+        if (strcmp(expected, str) != 0) {
+            fprintf(stderr, "error: expected '%s', but got '%s', which "
+                   "is not equal. %s\n", expected, str, text);
+            return -1;
+        }
+        break;
+    case TEST_ERROR_GE:
+        if (strcmp(expected, str) > 0) {
+            fprintf(stderr, "error: expected '%s', but got '%s'. "
+                   "Expected something greater or equal.  %s\n",
+                   expected, str, text);
+            return -1;
+        }
+        break;
+    case TEST_ERROR_GT:
+        if (strcmp(expected, str) >= 0) {
+            fprintf(stderr, "error: expected '%s', but got '%s'. "
+                   "Expected something greater.  %s\n",
+                   expected, str, text);
+            return -1;
+        }
+        break;
+    case TEST_ERROR_LE:
+        if (strcmp(expected, str) < 0) {
+            fprintf(stderr, "error: expected '%s', but got '%s'. "
+                   "Expected something less or equal.  %s\n",
+                   expected, str, text);
+            return -1;
+        }
+        break;
+    case TEST_ERROR_LT:
+        if (strcmp(expected, str) <= 0) {
+            fprintf(stderr, "error: expected '%s', but got '%s'. "
+                   "Expected something less.  %s\n",
+                   expected, str, text);
+            return -1;
+        }
+        break;
+    case TEST_ERROR_NE:
+        if (strcmp(expected, str) == 0) {
+            fprintf(stderr, "error: did not expect '%s': '%s'\n",
+                   expected, text);
+            return -1;
+        }
+        break;
+    default:
+        fprintf(stderr, "runtime error: invalid type %d passed in: '%s'\n",
+                ty, text);
+        return -1;
+    }
+    return 0;
+}
+
+int vexpect(const char *expected, const char *text, int ty,
+        const char *fmt, va_list ap)
+{
+    char *str = NULL;
+    int ret;
+
+    if (vasprintf(&str, fmt, ap) < 0) { // TODO: portability
+        fprintf(stderr, "error: vasprintf failed: %s\n", text);
+        return -1;
+    }
+    ret = vexpect_decode(expected, text, ty, str);
+    free(str);
+    return ret;
+}
+
+int expect(const char *expected, const char *text, int ty,
+        const char *fmt, ...)
+{
+    int ret;
+    va_list ap;
+
+    va_start(ap, fmt);
+    ret = vexpect(expected, text, ty, fmt, ap);
+    va_end(ap);
+    return ret;
+}
+
+void *xcalloc(size_t len)
+{
+    void *v = calloc(1, len);
+    if (!v) {
+        abort();
+    }
+    return v;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 154 - 0
hadoop-native-core/common/test.h

@@ -0,0 +1,154 @@
+/**
+ * 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 HADOOP_CORE_COMMON_TEST_H
+#define HADOOP_CORE_COMMON_TEST_H
+
+#include <inttypes.h> /* for PRIdPTR */
+#include <stdarg.h> /* for va_list */
+#include <unistd.h> /* for size_t */
+
+#define TEST_ERROR_EQ 0
+#define TEST_ERROR_GE 1
+#define TEST_ERROR_GT 2
+#define TEST_ERROR_LE 3
+#define TEST_ERROR_LT 4
+#define TEST_ERROR_NE 5
+
+/**
+ * Fail with an error message.
+ *
+ * @param fmt       printf-style format string.
+ * @param ...       printf-style arguments.
+ */
+void fail(const char *fmt, ...);
+
+/**
+ * Check for a test condition.
+ *
+ * @param expected  The string which is expected.
+ * @param text      Some text that will be printed out if the test
+ *                      condition fails.
+ * @param ty        Comparison type.  See TEST_ERROR_* constants.
+ * @param fmt       printf-style format string.
+ * @param ap        printf-style arguments.
+ *
+ * @return      0 on success; 1 on failure.
+ */
+int vexpect(const char *expected, const char *text, int ty,
+        const char *fmt, va_list ap);
+
+/**
+ * Check for a test condition.
+ *
+ * @param expected  The string which is expected.
+ * @param text      Some text that will be printed out if the test
+ *                      condition fails.
+ * @param ty        Comparison type.  See TEST_ERROR_* constants.
+ * @param fmt       printf-style format string.
+ * @param ...       printf-style arguments.
+ *
+ * @return          0 on success; 1 on failure.
+ */
+int expect(const char *expected, const char *text, int ty,
+        const char *fmt, ...);
+
+/**
+ * Allocate a zero-initialized region of memory, or die.
+ *
+ * @param len       The length
+ *
+ * @return          A pointer to a zero-initialized malloc'ed region.
+ */
+void *xcalloc(size_t len);
+
+#define TEST_ERROR_GET_LINE_HELPER2(x) #x
+#define TEST_ERROR_GET_LINE_HELPER(x) TEST_ERROR_GET_LINE_HELPER2(x)
+#define TEST_ERROR_LOCATION_TEXT __FILE__ " at line " \
+    TEST_ERROR_GET_LINE_HELPER(__LINE__)
+
+#define EXPECT(...) do { if (expect(__VA_ARGS__)) return 1; } while (0);
+
+#define EXPECT_EQ(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+           fmt, __VA_ARGS__);
+
+#define EXPECT_STR_EQ(expected, str) \
+    EXPECT_EQ(expected, "%s", str)
+
+#define EXPECT_GE(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_GE, \
+           fmt, __VA_ARGS__);
+
+#define EXPECT_GT(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_GT, \
+           fmt, __VA_ARGS__);
+
+#define EXPECT_LE(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_LE, \
+           fmt, __VA_ARGS__);
+
+#define EXPECT_LT(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_LT, \
+           fmt, __VA_ARGS__);
+
+#define EXPECT_NE(expected, fmt, ...) \
+    EXPECT(expected, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_NE, \
+           fmt, __VA_ARGS__);
+
+#define COMMON_TEST__TO_STR(x) #x
+#define COMMON_TEST__TO_STR2(x) COMMON_TEST__TO_STR(x)
+
+#define EXPECT_INT_EQ(expected, x) do { \
+  char expected_buf[16] = { 0 }; \
+  snprintf(expected_buf, sizeof(expected_buf), "%d", expected); \
+  EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, "%d", x); \
+} while(0);
+
+#define EXPECT_INT64_EQ(expected, x) do { \
+  char expected_buf[32] = { 0 }; \
+  snprintf(expected_buf, sizeof(expected_buf), "%" PRId64, expected); \
+  EXPECT(expected_buf, TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+         "%" PRId64, x); \
+} while(0);
+
+#define EXPECT_NO_HADOOP_ERR(expr) do { \
+  struct hadoop_err *err = expr; \
+  EXPECT("", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+         "%s", (err ? hadoop_err_msg(err) : "")); \
+} while(0);
+
+#define EXPECT_INT_ZERO(x) \
+    EXPECT("0", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+           "%d", x);
+
+#define EXPECT_TRUE(x) \
+    EXPECT("1", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+           "%d", (!!x));
+
+#define EXPECT_FALSE(x) \
+    EXPECT("0", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_EQ, \
+           "%d", (!!x));
+
+#define EXPECT_NONNULL(x) \
+    EXPECT("0", TEST_ERROR_LOCATION_TEXT, TEST_ERROR_NE, \
+           "%"PRIdPTR, x);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 765 - 0
hadoop-native-core/common/tree.h

@@ -0,0 +1,765 @@
+/*	$NetBSD: tree.h,v 1.8 2004/03/28 19:38:30 provos Exp $	*/
+/*	$OpenBSD: tree.h,v 1.7 2002/10/17 21:51:54 art Exp $	*/
+/* $FreeBSD$ */
+
+/*-
+ * Copyright 2002 Niels Provos <provos@citi.umich.edu>
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef	_SYS_TREE_H_
+#define	_SYS_TREE_H_
+
+#include <sys/cdefs.h>
+
+/*
+ * This file defines data structures for different types of trees:
+ * splay trees and red-black trees.
+ *
+ * A splay tree is a self-organizing data structure.  Every operation
+ * on the tree causes a splay to happen.  The splay moves the requested
+ * node to the root of the tree and partly rebalances it.
+ *
+ * This has the benefit that request locality causes faster lookups as
+ * the requested nodes move to the top of the tree.  On the other hand,
+ * every lookup causes memory writes.
+ *
+ * The Balance Theorem bounds the total access time for m operations
+ * and n inserts on an initially empty tree as O((m + n)lg n).  The
+ * amortized cost for a sequence of m accesses to a splay tree is O(lg n);
+ *
+ * A red-black tree is a binary search tree with the node color as an
+ * extra attribute.  It fulfills a set of conditions:
+ *	- every search path from the root to a leaf consists of the
+ *	  same number of black nodes,
+ *	- each red node (except for the root) has a black parent,
+ *	- each leaf node is black.
+ *
+ * Every operation on a red-black tree is bounded as O(lg n).
+ * The maximum height of a red-black tree is 2lg (n+1).
+ */
+
+#define SPLAY_HEAD(name, type)						\
+struct name {								\
+	struct type *sph_root; /* root of the tree */			\
+}
+
+#define SPLAY_INITIALIZER(root)						\
+	{ NULL }
+
+#define SPLAY_INIT(root) do {						\
+	(root)->sph_root = NULL;					\
+} while (/*CONSTCOND*/ 0)
+
+#define SPLAY_ENTRY(type)						\
+struct {								\
+	struct type *spe_left; /* left element */			\
+	struct type *spe_right; /* right element */			\
+}
+
+#define SPLAY_LEFT(elm, field)		(elm)->field.spe_left
+#define SPLAY_RIGHT(elm, field)		(elm)->field.spe_right
+#define SPLAY_ROOT(head)		(head)->sph_root
+#define SPLAY_EMPTY(head)		(SPLAY_ROOT(head) == NULL)
+
+/* SPLAY_ROTATE_{LEFT,RIGHT} expect that tmp hold SPLAY_{RIGHT,LEFT} */
+#define SPLAY_ROTATE_RIGHT(head, tmp, field) do {			\
+	SPLAY_LEFT((head)->sph_root, field) = SPLAY_RIGHT(tmp, field);	\
+	SPLAY_RIGHT(tmp, field) = (head)->sph_root;			\
+	(head)->sph_root = tmp;						\
+} while (/*CONSTCOND*/ 0)
+	
+#define SPLAY_ROTATE_LEFT(head, tmp, field) do {			\
+	SPLAY_RIGHT((head)->sph_root, field) = SPLAY_LEFT(tmp, field);	\
+	SPLAY_LEFT(tmp, field) = (head)->sph_root;			\
+	(head)->sph_root = tmp;						\
+} while (/*CONSTCOND*/ 0)
+
+#define SPLAY_LINKLEFT(head, tmp, field) do {				\
+	SPLAY_LEFT(tmp, field) = (head)->sph_root;			\
+	tmp = (head)->sph_root;						\
+	(head)->sph_root = SPLAY_LEFT((head)->sph_root, field);		\
+} while (/*CONSTCOND*/ 0)
+
+#define SPLAY_LINKRIGHT(head, tmp, field) do {				\
+	SPLAY_RIGHT(tmp, field) = (head)->sph_root;			\
+	tmp = (head)->sph_root;						\
+	(head)->sph_root = SPLAY_RIGHT((head)->sph_root, field);	\
+} while (/*CONSTCOND*/ 0)
+
+#define SPLAY_ASSEMBLE(head, node, left, right, field) do {		\
+	SPLAY_RIGHT(left, field) = SPLAY_LEFT((head)->sph_root, field);	\
+	SPLAY_LEFT(right, field) = SPLAY_RIGHT((head)->sph_root, field);\
+	SPLAY_LEFT((head)->sph_root, field) = SPLAY_RIGHT(node, field);	\
+	SPLAY_RIGHT((head)->sph_root, field) = SPLAY_LEFT(node, field);	\
+} while (/*CONSTCOND*/ 0)
+
+/* Generates prototypes and inline functions */
+
+#define SPLAY_PROTOTYPE(name, type, field, cmp)				\
+void name##_SPLAY(struct name *, struct type *);			\
+void name##_SPLAY_MINMAX(struct name *, int);				\
+struct type *name##_SPLAY_INSERT(struct name *, struct type *);		\
+struct type *name##_SPLAY_REMOVE(struct name *, struct type *);		\
+									\
+/* Finds the node with the same key as elm */				\
+static __inline struct type *						\
+name##_SPLAY_FIND(struct name *head, struct type *elm)			\
+{									\
+	if (SPLAY_EMPTY(head))						\
+		return(NULL);						\
+	name##_SPLAY(head, elm);					\
+	if ((cmp)(elm, (head)->sph_root) == 0)				\
+		return (head->sph_root);				\
+	return (NULL);							\
+}									\
+									\
+static __inline struct type *						\
+name##_SPLAY_NEXT(struct name *head, struct type *elm)			\
+{									\
+	name##_SPLAY(head, elm);					\
+	if (SPLAY_RIGHT(elm, field) != NULL) {				\
+		elm = SPLAY_RIGHT(elm, field);				\
+		while (SPLAY_LEFT(elm, field) != NULL) {		\
+			elm = SPLAY_LEFT(elm, field);			\
+		}							\
+	} else								\
+		elm = NULL;						\
+	return (elm);							\
+}									\
+									\
+static __inline struct type *						\
+name##_SPLAY_MIN_MAX(struct name *head, int val)			\
+{									\
+	name##_SPLAY_MINMAX(head, val);					\
+        return (SPLAY_ROOT(head));					\
+}
+
+/* Main splay operation.
+ * Moves node close to the key of elm to top
+ */
+#define SPLAY_GENERATE(name, type, field, cmp)				\
+struct type *								\
+name##_SPLAY_INSERT(struct name *head, struct type *elm)		\
+{									\
+    if (SPLAY_EMPTY(head)) {						\
+	    SPLAY_LEFT(elm, field) = SPLAY_RIGHT(elm, field) = NULL;	\
+    } else {								\
+	    int __comp;							\
+	    name##_SPLAY(head, elm);					\
+	    __comp = (cmp)(elm, (head)->sph_root);			\
+	    if(__comp < 0) {						\
+		    SPLAY_LEFT(elm, field) = SPLAY_LEFT((head)->sph_root, field);\
+		    SPLAY_RIGHT(elm, field) = (head)->sph_root;		\
+		    SPLAY_LEFT((head)->sph_root, field) = NULL;		\
+	    } else if (__comp > 0) {					\
+		    SPLAY_RIGHT(elm, field) = SPLAY_RIGHT((head)->sph_root, field);\
+		    SPLAY_LEFT(elm, field) = (head)->sph_root;		\
+		    SPLAY_RIGHT((head)->sph_root, field) = NULL;	\
+	    } else							\
+		    return ((head)->sph_root);				\
+    }									\
+    (head)->sph_root = (elm);						\
+    return (NULL);							\
+}									\
+									\
+struct type *								\
+name##_SPLAY_REMOVE(struct name *head, struct type *elm)		\
+{									\
+	struct type *__tmp;						\
+	if (SPLAY_EMPTY(head))						\
+		return (NULL);						\
+	name##_SPLAY(head, elm);					\
+	if ((cmp)(elm, (head)->sph_root) == 0) {			\
+		if (SPLAY_LEFT((head)->sph_root, field) == NULL) {	\
+			(head)->sph_root = SPLAY_RIGHT((head)->sph_root, field);\
+		} else {						\
+			__tmp = SPLAY_RIGHT((head)->sph_root, field);	\
+			(head)->sph_root = SPLAY_LEFT((head)->sph_root, field);\
+			name##_SPLAY(head, elm);			\
+			SPLAY_RIGHT((head)->sph_root, field) = __tmp;	\
+		}							\
+		return (elm);						\
+	}								\
+	return (NULL);							\
+}									\
+									\
+void									\
+name##_SPLAY(struct name *head, struct type *elm)			\
+{									\
+	struct type __node, *__left, *__right, *__tmp;			\
+	int __comp;							\
+\
+	SPLAY_LEFT(&__node, field) = SPLAY_RIGHT(&__node, field) = NULL;\
+	__left = __right = &__node;					\
+\
+	while ((__comp = (cmp)(elm, (head)->sph_root)) != 0) {		\
+		if (__comp < 0) {					\
+			__tmp = SPLAY_LEFT((head)->sph_root, field);	\
+			if (__tmp == NULL)				\
+				break;					\
+			if ((cmp)(elm, __tmp) < 0){			\
+				SPLAY_ROTATE_RIGHT(head, __tmp, field);	\
+				if (SPLAY_LEFT((head)->sph_root, field) == NULL)\
+					break;				\
+			}						\
+			SPLAY_LINKLEFT(head, __right, field);		\
+		} else if (__comp > 0) {				\
+			__tmp = SPLAY_RIGHT((head)->sph_root, field);	\
+			if (__tmp == NULL)				\
+				break;					\
+			if ((cmp)(elm, __tmp) > 0){			\
+				SPLAY_ROTATE_LEFT(head, __tmp, field);	\
+				if (SPLAY_RIGHT((head)->sph_root, field) == NULL)\
+					break;				\
+			}						\
+			SPLAY_LINKRIGHT(head, __left, field);		\
+		}							\
+	}								\
+	SPLAY_ASSEMBLE(head, &__node, __left, __right, field);		\
+}									\
+									\
+/* Splay with either the minimum or the maximum element			\
+ * Used to find minimum or maximum element in tree.			\
+ */									\
+void name##_SPLAY_MINMAX(struct name *head, int __comp) \
+{									\
+	struct type __node, *__left, *__right, *__tmp;			\
+\
+	SPLAY_LEFT(&__node, field) = SPLAY_RIGHT(&__node, field) = NULL;\
+	__left = __right = &__node;					\
+\
+	while (1) {							\
+		if (__comp < 0) {					\
+			__tmp = SPLAY_LEFT((head)->sph_root, field);	\
+			if (__tmp == NULL)				\
+				break;					\
+			if (__comp < 0){				\
+				SPLAY_ROTATE_RIGHT(head, __tmp, field);	\
+				if (SPLAY_LEFT((head)->sph_root, field) == NULL)\
+					break;				\
+			}						\
+			SPLAY_LINKLEFT(head, __right, field);		\
+		} else if (__comp > 0) {				\
+			__tmp = SPLAY_RIGHT((head)->sph_root, field);	\
+			if (__tmp == NULL)				\
+				break;					\
+			if (__comp > 0) {				\
+				SPLAY_ROTATE_LEFT(head, __tmp, field);	\
+				if (SPLAY_RIGHT((head)->sph_root, field) == NULL)\
+					break;				\
+			}						\
+			SPLAY_LINKRIGHT(head, __left, field);		\
+		}							\
+	}								\
+	SPLAY_ASSEMBLE(head, &__node, __left, __right, field);		\
+}
+
+#define SPLAY_NEGINF	-1
+#define SPLAY_INF	1
+
+#define SPLAY_INSERT(name, x, y)	name##_SPLAY_INSERT(x, y)
+#define SPLAY_REMOVE(name, x, y)	name##_SPLAY_REMOVE(x, y)
+#define SPLAY_FIND(name, x, y)		name##_SPLAY_FIND(x, y)
+#define SPLAY_NEXT(name, x, y)		name##_SPLAY_NEXT(x, y)
+#define SPLAY_MIN(name, x)		(SPLAY_EMPTY(x) ? NULL	\
+					: name##_SPLAY_MIN_MAX(x, SPLAY_NEGINF))
+#define SPLAY_MAX(name, x)		(SPLAY_EMPTY(x) ? NULL	\
+					: name##_SPLAY_MIN_MAX(x, SPLAY_INF))
+
+#define SPLAY_FOREACH(x, name, head)					\
+	for ((x) = SPLAY_MIN(name, head);				\
+	     (x) != NULL;						\
+	     (x) = SPLAY_NEXT(name, head, x))
+
+/* Macros that define a red-black tree */
+#define RB_HEAD(name, type)						\
+struct name {								\
+	struct type *rbh_root; /* root of the tree */			\
+}
+
+#define RB_INITIALIZER(root)						\
+	{ NULL }
+
+#define RB_INIT(root) do {						\
+	(root)->rbh_root = NULL;					\
+} while (/*CONSTCOND*/ 0)
+
+#define RB_BLACK	0
+#define RB_RED		1
+#define RB_ENTRY(type)							\
+struct {								\
+	struct type *rbe_left;		/* left element */		\
+	struct type *rbe_right;		/* right element */		\
+	struct type *rbe_parent;	/* parent element */		\
+	int rbe_color;			/* node color */		\
+}
+
+#define RB_LEFT(elm, field)		(elm)->field.rbe_left
+#define RB_RIGHT(elm, field)		(elm)->field.rbe_right
+#define RB_PARENT(elm, field)		(elm)->field.rbe_parent
+#define RB_COLOR(elm, field)		(elm)->field.rbe_color
+#define RB_ROOT(head)			(head)->rbh_root
+#define RB_EMPTY(head)			(RB_ROOT(head) == NULL)
+
+#define RB_SET(elm, parent, field) do {					\
+	RB_PARENT(elm, field) = parent;					\
+	RB_LEFT(elm, field) = RB_RIGHT(elm, field) = NULL;		\
+	RB_COLOR(elm, field) = RB_RED;					\
+} while (/*CONSTCOND*/ 0)
+
+#define RB_SET_BLACKRED(black, red, field) do {				\
+	RB_COLOR(black, field) = RB_BLACK;				\
+	RB_COLOR(red, field) = RB_RED;					\
+} while (/*CONSTCOND*/ 0)
+
+#ifndef RB_AUGMENT
+#define RB_AUGMENT(x)	do {} while (0)
+#endif
+
+#define RB_ROTATE_LEFT(head, elm, tmp, field) do {			\
+	(tmp) = RB_RIGHT(elm, field);					\
+	if ((RB_RIGHT(elm, field) = RB_LEFT(tmp, field)) != NULL) {	\
+		RB_PARENT(RB_LEFT(tmp, field), field) = (elm);		\
+	}								\
+	RB_AUGMENT(elm);						\
+	if ((RB_PARENT(tmp, field) = RB_PARENT(elm, field)) != NULL) {	\
+		if ((elm) == RB_LEFT(RB_PARENT(elm, field), field))	\
+			RB_LEFT(RB_PARENT(elm, field), field) = (tmp);	\
+		else							\
+			RB_RIGHT(RB_PARENT(elm, field), field) = (tmp);	\
+	} else								\
+		(head)->rbh_root = (tmp);				\
+	RB_LEFT(tmp, field) = (elm);					\
+	RB_PARENT(elm, field) = (tmp);					\
+	RB_AUGMENT(tmp);						\
+	if ((RB_PARENT(tmp, field)))					\
+		RB_AUGMENT(RB_PARENT(tmp, field));			\
+} while (/*CONSTCOND*/ 0)
+
+#define RB_ROTATE_RIGHT(head, elm, tmp, field) do {			\
+	(tmp) = RB_LEFT(elm, field);					\
+	if ((RB_LEFT(elm, field) = RB_RIGHT(tmp, field)) != NULL) {	\
+		RB_PARENT(RB_RIGHT(tmp, field), field) = (elm);		\
+	}								\
+	RB_AUGMENT(elm);						\
+	if ((RB_PARENT(tmp, field) = RB_PARENT(elm, field)) != NULL) {	\
+		if ((elm) == RB_LEFT(RB_PARENT(elm, field), field))	\
+			RB_LEFT(RB_PARENT(elm, field), field) = (tmp);	\
+		else							\
+			RB_RIGHT(RB_PARENT(elm, field), field) = (tmp);	\
+	} else								\
+		(head)->rbh_root = (tmp);				\
+	RB_RIGHT(tmp, field) = (elm);					\
+	RB_PARENT(elm, field) = (tmp);					\
+	RB_AUGMENT(tmp);						\
+	if ((RB_PARENT(tmp, field)))					\
+		RB_AUGMENT(RB_PARENT(tmp, field));			\
+} while (/*CONSTCOND*/ 0)
+
+/* Generates prototypes and inline functions */
+#define	RB_PROTOTYPE(name, type, field, cmp)				\
+	RB_PROTOTYPE_INTERNAL(name, type, field, cmp,)
+#define	RB_PROTOTYPE_STATIC(name, type, field, cmp)			\
+	RB_PROTOTYPE_INTERNAL(name, type, field, cmp, __unused static)
+#define RB_PROTOTYPE_INTERNAL(name, type, field, cmp, attr)		\
+attr void name##_RB_INSERT_COLOR(struct name *, struct type *);		\
+attr void name##_RB_REMOVE_COLOR(struct name *, struct type *, struct type *);\
+attr struct type *name##_RB_REMOVE(struct name *, struct type *);	\
+attr struct type *name##_RB_INSERT(struct name *, struct type *);	\
+attr struct type *name##_RB_FIND(struct name *, struct type *);		\
+attr struct type *name##_RB_NFIND(struct name *, struct type *);	\
+attr struct type *name##_RB_NEXT(struct type *);			\
+attr struct type *name##_RB_PREV(struct type *);			\
+attr struct type *name##_RB_MINMAX(struct name *, int);			\
+									\
+
+/* Main rb operation.
+ * Moves node close to the key of elm to top
+ */
+#define	RB_GENERATE(name, type, field, cmp)				\
+	RB_GENERATE_INTERNAL(name, type, field, cmp,)
+#define	RB_GENERATE_STATIC(name, type, field, cmp)			\
+	RB_GENERATE_INTERNAL(name, type, field, cmp, __unused static)
+#define RB_GENERATE_INTERNAL(name, type, field, cmp, attr)		\
+attr void								\
+name##_RB_INSERT_COLOR(struct name *head, struct type *elm)		\
+{									\
+	struct type *parent, *gparent, *tmp;				\
+	while ((parent = RB_PARENT(elm, field)) != NULL &&		\
+	    RB_COLOR(parent, field) == RB_RED) {			\
+		gparent = RB_PARENT(parent, field);			\
+		if (parent == RB_LEFT(gparent, field)) {		\
+			tmp = RB_RIGHT(gparent, field);			\
+			if (tmp && RB_COLOR(tmp, field) == RB_RED) {	\
+				RB_COLOR(tmp, field) = RB_BLACK;	\
+				RB_SET_BLACKRED(parent, gparent, field);\
+				elm = gparent;				\
+				continue;				\
+			}						\
+			if (RB_RIGHT(parent, field) == elm) {		\
+				RB_ROTATE_LEFT(head, parent, tmp, field);\
+				tmp = parent;				\
+				parent = elm;				\
+				elm = tmp;				\
+			}						\
+			RB_SET_BLACKRED(parent, gparent, field);	\
+			RB_ROTATE_RIGHT(head, gparent, tmp, field);	\
+		} else {						\
+			tmp = RB_LEFT(gparent, field);			\
+			if (tmp && RB_COLOR(tmp, field) == RB_RED) {	\
+				RB_COLOR(tmp, field) = RB_BLACK;	\
+				RB_SET_BLACKRED(parent, gparent, field);\
+				elm = gparent;				\
+				continue;				\
+			}						\
+			if (RB_LEFT(parent, field) == elm) {		\
+				RB_ROTATE_RIGHT(head, parent, tmp, field);\
+				tmp = parent;				\
+				parent = elm;				\
+				elm = tmp;				\
+			}						\
+			RB_SET_BLACKRED(parent, gparent, field);	\
+			RB_ROTATE_LEFT(head, gparent, tmp, field);	\
+		}							\
+	}								\
+	RB_COLOR(head->rbh_root, field) = RB_BLACK;			\
+}									\
+									\
+attr void								\
+name##_RB_REMOVE_COLOR(struct name *head, struct type *parent, struct type *elm) \
+{									\
+	struct type *tmp;						\
+	while ((elm == NULL || RB_COLOR(elm, field) == RB_BLACK) &&	\
+	    elm != RB_ROOT(head)) {					\
+		if (RB_LEFT(parent, field) == elm) {			\
+			tmp = RB_RIGHT(parent, field);			\
+			if (RB_COLOR(tmp, field) == RB_RED) {		\
+				RB_SET_BLACKRED(tmp, parent, field);	\
+				RB_ROTATE_LEFT(head, parent, tmp, field);\
+				tmp = RB_RIGHT(parent, field);		\
+			}						\
+			if ((RB_LEFT(tmp, field) == NULL ||		\
+			    RB_COLOR(RB_LEFT(tmp, field), field) == RB_BLACK) &&\
+			    (RB_RIGHT(tmp, field) == NULL ||		\
+			    RB_COLOR(RB_RIGHT(tmp, field), field) == RB_BLACK)) {\
+				RB_COLOR(tmp, field) = RB_RED;		\
+				elm = parent;				\
+				parent = RB_PARENT(elm, field);		\
+			} else {					\
+				if (RB_RIGHT(tmp, field) == NULL ||	\
+				    RB_COLOR(RB_RIGHT(tmp, field), field) == RB_BLACK) {\
+					struct type *oleft;		\
+					if ((oleft = RB_LEFT(tmp, field)) \
+					    != NULL)			\
+						RB_COLOR(oleft, field) = RB_BLACK;\
+					RB_COLOR(tmp, field) = RB_RED;	\
+					RB_ROTATE_RIGHT(head, tmp, oleft, field);\
+					tmp = RB_RIGHT(parent, field);	\
+				}					\
+				RB_COLOR(tmp, field) = RB_COLOR(parent, field);\
+				RB_COLOR(parent, field) = RB_BLACK;	\
+				if (RB_RIGHT(tmp, field))		\
+					RB_COLOR(RB_RIGHT(tmp, field), field) = RB_BLACK;\
+				RB_ROTATE_LEFT(head, parent, tmp, field);\
+				elm = RB_ROOT(head);			\
+				break;					\
+			}						\
+		} else {						\
+			tmp = RB_LEFT(parent, field);			\
+			if (RB_COLOR(tmp, field) == RB_RED) {		\
+				RB_SET_BLACKRED(tmp, parent, field);	\
+				RB_ROTATE_RIGHT(head, parent, tmp, field);\
+				tmp = RB_LEFT(parent, field);		\
+			}						\
+			if ((RB_LEFT(tmp, field) == NULL ||		\
+			    RB_COLOR(RB_LEFT(tmp, field), field) == RB_BLACK) &&\
+			    (RB_RIGHT(tmp, field) == NULL ||		\
+			    RB_COLOR(RB_RIGHT(tmp, field), field) == RB_BLACK)) {\
+				RB_COLOR(tmp, field) = RB_RED;		\
+				elm = parent;				\
+				parent = RB_PARENT(elm, field);		\
+			} else {					\
+				if (RB_LEFT(tmp, field) == NULL ||	\
+				    RB_COLOR(RB_LEFT(tmp, field), field) == RB_BLACK) {\
+					struct type *oright;		\
+					if ((oright = RB_RIGHT(tmp, field)) \
+					    != NULL)			\
+						RB_COLOR(oright, field) = RB_BLACK;\
+					RB_COLOR(tmp, field) = RB_RED;	\
+					RB_ROTATE_LEFT(head, tmp, oright, field);\
+					tmp = RB_LEFT(parent, field);	\
+				}					\
+				RB_COLOR(tmp, field) = RB_COLOR(parent, field);\
+				RB_COLOR(parent, field) = RB_BLACK;	\
+				if (RB_LEFT(tmp, field))		\
+					RB_COLOR(RB_LEFT(tmp, field), field) = RB_BLACK;\
+				RB_ROTATE_RIGHT(head, parent, tmp, field);\
+				elm = RB_ROOT(head);			\
+				break;					\
+			}						\
+		}							\
+	}								\
+	if (elm)							\
+		RB_COLOR(elm, field) = RB_BLACK;			\
+}									\
+									\
+attr struct type *							\
+name##_RB_REMOVE(struct name *head, struct type *elm)			\
+{									\
+	struct type *child, *parent, *old = elm;			\
+	int color;							\
+	if (RB_LEFT(elm, field) == NULL)				\
+		child = RB_RIGHT(elm, field);				\
+	else if (RB_RIGHT(elm, field) == NULL)				\
+		child = RB_LEFT(elm, field);				\
+	else {								\
+		struct type *left;					\
+		elm = RB_RIGHT(elm, field);				\
+		while ((left = RB_LEFT(elm, field)) != NULL)		\
+			elm = left;					\
+		child = RB_RIGHT(elm, field);				\
+		parent = RB_PARENT(elm, field);				\
+		color = RB_COLOR(elm, field);				\
+		if (child)						\
+			RB_PARENT(child, field) = parent;		\
+		if (parent) {						\
+			if (RB_LEFT(parent, field) == elm)		\
+				RB_LEFT(parent, field) = child;		\
+			else						\
+				RB_RIGHT(parent, field) = child;	\
+			RB_AUGMENT(parent);				\
+		} else							\
+			RB_ROOT(head) = child;				\
+		if (RB_PARENT(elm, field) == old)			\
+			parent = elm;					\
+		(elm)->field = (old)->field;				\
+		if (RB_PARENT(old, field)) {				\
+			if (RB_LEFT(RB_PARENT(old, field), field) == old)\
+				RB_LEFT(RB_PARENT(old, field), field) = elm;\
+			else						\
+				RB_RIGHT(RB_PARENT(old, field), field) = elm;\
+			RB_AUGMENT(RB_PARENT(old, field));		\
+		} else							\
+			RB_ROOT(head) = elm;				\
+		RB_PARENT(RB_LEFT(old, field), field) = elm;		\
+		if (RB_RIGHT(old, field))				\
+			RB_PARENT(RB_RIGHT(old, field), field) = elm;	\
+		if (parent) {						\
+			left = parent;					\
+			do {						\
+				RB_AUGMENT(left);			\
+			} while ((left = RB_PARENT(left, field)) != NULL); \
+		}							\
+		goto color;						\
+	}								\
+	parent = RB_PARENT(elm, field);					\
+	color = RB_COLOR(elm, field);					\
+	if (child)							\
+		RB_PARENT(child, field) = parent;			\
+	if (parent) {							\
+		if (RB_LEFT(parent, field) == elm)			\
+			RB_LEFT(parent, field) = child;			\
+		else							\
+			RB_RIGHT(parent, field) = child;		\
+		RB_AUGMENT(parent);					\
+	} else								\
+		RB_ROOT(head) = child;					\
+color:									\
+	if (color == RB_BLACK)						\
+		name##_RB_REMOVE_COLOR(head, parent, child);		\
+	return (old);							\
+}									\
+									\
+/* Inserts a node into the RB tree */					\
+attr struct type *							\
+name##_RB_INSERT(struct name *head, struct type *elm)			\
+{									\
+	struct type *tmp;						\
+	struct type *parent = NULL;					\
+	int comp = 0;							\
+	tmp = RB_ROOT(head);						\
+	while (tmp) {							\
+		parent = tmp;						\
+		comp = (cmp)(elm, parent);				\
+		if (comp < 0)						\
+			tmp = RB_LEFT(tmp, field);			\
+		else if (comp > 0)					\
+			tmp = RB_RIGHT(tmp, field);			\
+		else							\
+			return (tmp);					\
+	}								\
+	RB_SET(elm, parent, field);					\
+	if (parent != NULL) {						\
+		if (comp < 0)						\
+			RB_LEFT(parent, field) = elm;			\
+		else							\
+			RB_RIGHT(parent, field) = elm;			\
+		RB_AUGMENT(parent);					\
+	} else								\
+		RB_ROOT(head) = elm;					\
+	name##_RB_INSERT_COLOR(head, elm);				\
+	return (NULL);							\
+}									\
+									\
+/* Finds the node with the same key as elm */				\
+attr struct type *							\
+name##_RB_FIND(struct name *head, struct type *elm)			\
+{									\
+	struct type *tmp = RB_ROOT(head);				\
+	int comp;							\
+	while (tmp) {							\
+		comp = cmp(elm, tmp);					\
+		if (comp < 0)						\
+			tmp = RB_LEFT(tmp, field);			\
+		else if (comp > 0)					\
+			tmp = RB_RIGHT(tmp, field);			\
+		else							\
+			return (tmp);					\
+	}								\
+	return (NULL);							\
+}									\
+									\
+/* Finds the first node greater than or equal to the search key */	\
+attr struct type *							\
+name##_RB_NFIND(struct name *head, struct type *elm)			\
+{									\
+	struct type *tmp = RB_ROOT(head);				\
+	struct type *res = NULL;					\
+	int comp;							\
+	while (tmp) {							\
+		comp = cmp(elm, tmp);					\
+		if (comp < 0) {						\
+			res = tmp;					\
+			tmp = RB_LEFT(tmp, field);			\
+		}							\
+		else if (comp > 0)					\
+			tmp = RB_RIGHT(tmp, field);			\
+		else							\
+			return (tmp);					\
+	}								\
+	return (res);							\
+}									\
+									\
+/* ARGSUSED */								\
+attr struct type *							\
+name##_RB_NEXT(struct type *elm)					\
+{									\
+	if (RB_RIGHT(elm, field)) {					\
+		elm = RB_RIGHT(elm, field);				\
+		while (RB_LEFT(elm, field))				\
+			elm = RB_LEFT(elm, field);			\
+	} else {							\
+		if (RB_PARENT(elm, field) &&				\
+		    (elm == RB_LEFT(RB_PARENT(elm, field), field)))	\
+			elm = RB_PARENT(elm, field);			\
+		else {							\
+			while (RB_PARENT(elm, field) &&			\
+			    (elm == RB_RIGHT(RB_PARENT(elm, field), field)))\
+				elm = RB_PARENT(elm, field);		\
+			elm = RB_PARENT(elm, field);			\
+		}							\
+	}								\
+	return (elm);							\
+}									\
+									\
+/* ARGSUSED */								\
+attr struct type *							\
+name##_RB_PREV(struct type *elm)					\
+{									\
+	if (RB_LEFT(elm, field)) {					\
+		elm = RB_LEFT(elm, field);				\
+		while (RB_RIGHT(elm, field))				\
+			elm = RB_RIGHT(elm, field);			\
+	} else {							\
+		if (RB_PARENT(elm, field) &&				\
+		    (elm == RB_RIGHT(RB_PARENT(elm, field), field)))	\
+			elm = RB_PARENT(elm, field);			\
+		else {							\
+			while (RB_PARENT(elm, field) &&			\
+			    (elm == RB_LEFT(RB_PARENT(elm, field), field)))\
+				elm = RB_PARENT(elm, field);		\
+			elm = RB_PARENT(elm, field);			\
+		}							\
+	}								\
+	return (elm);							\
+}									\
+									\
+attr struct type *							\
+name##_RB_MINMAX(struct name *head, int val)				\
+{									\
+	struct type *tmp = RB_ROOT(head);				\
+	struct type *parent = NULL;					\
+	while (tmp) {							\
+		parent = tmp;						\
+		if (val < 0)						\
+			tmp = RB_LEFT(tmp, field);			\
+		else							\
+			tmp = RB_RIGHT(tmp, field);			\
+	}								\
+	return (parent);						\
+}
+
+#define RB_NEGINF	-1
+#define RB_INF	1
+
+#define RB_INSERT(name, x, y)	name##_RB_INSERT(x, y)
+#define RB_REMOVE(name, x, y)	name##_RB_REMOVE(x, y)
+#define RB_FIND(name, x, y)	name##_RB_FIND(x, y)
+#define RB_NFIND(name, x, y)	name##_RB_NFIND(x, y)
+#define RB_NEXT(name, x, y)	name##_RB_NEXT(y)
+#define RB_PREV(name, x, y)	name##_RB_PREV(y)
+#define RB_MIN(name, x)		name##_RB_MINMAX(x, RB_NEGINF)
+#define RB_MAX(name, x)		name##_RB_MINMAX(x, RB_INF)
+
+#define RB_FOREACH(x, name, head)					\
+	for ((x) = RB_MIN(name, head);					\
+	     (x) != NULL;						\
+	     (x) = name##_RB_NEXT(x))
+
+#define RB_FOREACH_FROM(x, name, y)					\
+	for ((x) = (y);							\
+	    ((x) != NULL) && ((y) = name##_RB_NEXT(x), (x) != NULL);	\
+	     (x) = (y))
+
+#define RB_FOREACH_SAFE(x, name, head, y)				\
+	for ((x) = RB_MIN(name, head);					\
+	    ((x) != NULL) && ((y) = name##_RB_NEXT(x), (x) != NULL);	\
+	     (x) = (y))
+
+#define RB_FOREACH_REVERSE(x, name, head)				\
+	for ((x) = RB_MAX(name, head);					\
+	     (x) != NULL;						\
+	     (x) = name##_RB_PREV(x))
+
+#define RB_FOREACH_REVERSE_FROM(x, name, y)				\
+	for ((x) = (y);							\
+	    ((x) != NULL) && ((y) = name##_RB_PREV(x), (x) != NULL);	\
+	     (x) = (y))
+
+#define RB_FOREACH_REVERSE_SAFE(x, name, head, y)			\
+	for ((x) = RB_MAX(name, head);					\
+	    ((x) != NULL) && ((y) = name##_RB_PREV(x), (x) != NULL);	\
+	     (x) = (y))
+
+#endif	/* _SYS_TREE_H_ */

+ 123 - 0
hadoop-native-core/hdfs/namenode-rpc-unit.c

@@ -0,0 +1,123 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "common/test.h"
+#include "protobuf/ClientNamenodeProtocol.call.h"
+#include "rpc/messenger.h"
+#include "rpc/proxy.h"
+
+#include <netinet/in.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <strings.h>
+#include <uv.h>
+
+struct options {
+    struct sockaddr_in remote;
+};
+
+static void options_from_env(struct options *opts)
+{
+    const char *ip_str;
+    const char *port_str;
+    int res, port;
+
+    ip_str = getenv("HDFS_IP");
+    if (!ip_str) {
+        fprintf(stderr, "You must set an ip via the HDFS_IP "
+                "environment variable.\n");
+        exit(EXIT_FAILURE);
+    }
+    port_str = getenv("HDFS_PORT");
+    if (!port_str) {
+        fprintf(stderr, "You must set a port via the HDFS_PORT "
+                "environment variable.\n");
+        exit(EXIT_FAILURE);
+    }
+    port = atoi(port_str);
+    res = uv_ip4_addr(ip_str, port, &opts->remote);
+    if (res) {
+        fprintf(stderr, "Invalid IP and port %s and %d: error %s\n",
+                ip_str, port, uv_strerror(res));
+        exit(EXIT_FAILURE);
+    }
+}
+
+void set_replication_cb(SetReplicationResponseProto *resp,
+                        struct hadoop_err *err, void *cb_data)
+{
+    uv_sem_t *sem = cb_data;
+
+    if (err) {
+        fprintf(stderr, "set_replication_cb: got an error.  %s\n",
+                hadoop_err_msg(err));
+        goto done;
+    }
+    fprintf(stderr, "set_replication_cb: resp->result = %d\n",
+            !!resp->result);
+
+done:
+    sem_post(sem);
+    if (err) {
+        hadoop_err_free(err);
+    }
+    if (resp) {
+        set_replication_response_proto__free_unpacked(resp, NULL);
+    }
+}
+
+int main(void)
+{
+    struct hrpc_messenger_builder *msgr_bld;
+    struct hrpc_proxy_builder *proxy_bld;
+    struct hrpc_proxy *proxy;
+    struct hrpc_messenger *msgr;
+    struct options opts;
+    uv_sem_t sem;
+
+    memset(&opts, 0, sizeof(opts));
+    options_from_env(&opts);
+    msgr_bld = hrpc_messenger_builder_alloc();
+    EXPECT_NONNULL(msgr_bld);
+    EXPECT_NO_HADOOP_ERR(hrpc_messenger_create(msgr_bld, &msgr));
+    proxy_bld = hrpc_proxy_builder_alloc(msgr);
+    EXPECT_NONNULL(proxy_bld);
+    hrpc_proxy_builder_set_remote(proxy_bld, &opts.remote);
+    hrpc_proxy_builder_set_protocol(proxy_bld,
+                "org.apache.hadoop.hdfs.protocol.ClientProtocol");
+    EXPECT_NO_HADOOP_ERR(hrpc_proxy_create(proxy_bld, &proxy));
+    EXPECT_INT_ZERO(uv_sem_init(&sem, 0));
+    {
+        SetReplicationRequestProto req = SET_REPLICATION_REQUEST_PROTO__INIT;
+        req.src = "/foo2";
+        req.replication = 2;
+        cnn_async_set_replication(proxy, &req, set_replication_cb, &sem);
+    }
+    sem_wait(&sem);
+
+    hrpc_proxy_free(proxy);
+    hrpc_messenger_shutdown(msgr);
+    hrpc_messenger_free(msgr);
+    uv_sem_destroy(&sem);
+
+    return EXIT_SUCCESS;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 61 - 0
hadoop-native-core/rpc/call.c

@@ -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.
+ */
+
+#include "rpc/call.h"
+#include "rpc/proxy.h"
+
+#include <stdlib.h>
+#include <string.h>
+#include <uv.h>
+
+void hrpc_call_deliver_err(struct hrpc_call *call, struct hadoop_err *err)
+{
+    hrpc_raw_cb_t cb = call->cb;
+    void *cb_data = call->cb_data;
+    free(call->payload.base);
+    memset(call, 0, sizeof(*call));
+    __sync_fetch_and_or(&call->active, 0);
+    cb(NULL, err, cb_data);
+}
+
+void hrpc_call_deliver_resp(struct hrpc_call *call, struct hrpc_response *resp)
+{
+    hrpc_raw_cb_t cb = call->cb;
+    void *cb_data = call->cb_data;
+    free(call->payload.base);
+    memset(call, 0, sizeof(*call));
+    __sync_fetch_and_or(&call->active, 0);
+    cb(resp, NULL, cb_data);
+}
+
+int hrpc_call_activate(struct hrpc_call *call)
+{
+    return !!__sync_bool_compare_and_swap(&call->active, 0, 1);
+}
+
+void hrpc_call_deactivate(struct hrpc_call *call)
+{
+    __sync_fetch_and_and(&call->active, 0);
+}
+
+int hrpc_call_is_active(const struct hrpc_call *call)
+{
+    return !!__sync_fetch_and_or((int*)&call->active, 0);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 129 - 0
hadoop-native-core/rpc/call.h

@@ -0,0 +1,129 @@
+/**
+ * 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 HADOOP_CORE_RPC_CALL_H
+#define HADOOP_CORE_RPC_CALL_H
+
+#include "common/queue.h"
+#include "rpc/proxy.h" // for hrpc_raw_cb_t 
+
+#include <netinet/in.h> // for struct sockaddr_in
+#include <stdint.h> // for int32_t
+#include <uv.h> // for uv_buf_t
+
+/**
+ * The Hadoop call.
+ *
+ * Note: this is an internal header which users of the RPC layer don't need to
+ * include.
+ */
+
+struct hrpc_call;
+
+/**
+ * A Hadoop RPC call.
+ */
+struct hrpc_call {
+    /**
+     * Pointers used to put this call into an implicit linked list.
+     */
+    STAILQ_ENTRY(hrpc_call) entry;
+
+    /**
+     * Remote address we're sending to.
+     */
+    struct sockaddr_in remote;
+
+    /**
+     * The callback to make from the reactor thread when this call completes or
+     * hits an error.
+     */
+    hrpc_raw_cb_t cb;
+
+    /**
+     * The data to pass to the callback.
+     */
+    void *cb_data;
+
+    /**
+     * Malloc'ed payload to send.  Malloced.
+     */
+    uv_buf_t payload;
+
+    /**
+     * The Java protocol we're using.  Malloced.
+     */
+    char *protocol;
+
+    /**
+     * Nonzero if the call is currently active.  Must be set using atomic
+     * operations.
+     */
+    int active;
+};
+
+STAILQ_HEAD(hrpc_calls, hrpc_call);
+
+/**
+ * Activate this call.
+ *
+ * @param call          The call.
+ *
+ * @return              1 if the call was activated; 0 if it was already
+ *                          active.
+ */
+int hrpc_call_activate(struct hrpc_call *call);
+
+/**
+ * Deactivate this call.
+ *
+ * @param call          The call.
+ */
+void hrpc_call_deactivate(struct hrpc_call *call);
+
+/**
+ * Determine if the call is active.
+ *
+ * @param call          The call.
+ *
+ * @return              1 if the call is active; 0 if it is not.
+ */
+int hrpc_call_is_active(const struct hrpc_call *call);
+
+/**
+ * Deliver an error message.  Will free call->payload and zero all fields.
+ *
+ * @param call          The call.
+ * @param resp          The error to pass to the callback.  The callback
+ *                        is responsible for freeing this error.
+ */
+void hrpc_call_deliver_err(struct hrpc_call *call, struct hadoop_err *err);
+
+/**
+ * Deliver a response.  Will free call->payload and zero all fields.
+ *
+ * @param call          The call.
+ * @param resp          Malloc'ed data to pass to the callback.  The callback
+ *                        is responsible for freeing this data.
+ */
+void hrpc_call_deliver_resp(struct hrpc_call *call,
+                            struct hrpc_response *resp);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 71 - 0
hadoop-native-core/rpc/client_id.c

@@ -0,0 +1,71 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "rpc/client_id.h"
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void hrpc_client_id_generate_rand(struct hrpc_client_id *id)
+{
+    int i;
+
+    for (i = 0; i <= HRPC_CLIENT_ID_LEN - 3; i += 3) {
+        long int r = lrand48();
+        id->buf[i + 0] = r & 0xff;
+        id->buf[i + 1] = (r >> 8) & 0xff;
+        id->buf[i + 2] = (r >> 16) & 0xff;
+    }
+    id->buf[HRPC_CLIENT_ID_LEN - 2] = lrand48() & 0xff;
+    id->buf[HRPC_CLIENT_ID_LEN - 1] = lrand48() & 0xff;
+}
+
+struct hadoop_err *hrpc_client_id_from_bytes(void *bytes, int len,
+                                             struct hrpc_client_id *out)
+{
+    if (len != HRPC_CLIENT_ID_LEN) {
+        return hadoop_lerr_alloc(EINVAL, "hrpc_client_id_from_bytes: "
+                "invalid client id length of %d (expected %d)",
+                len, HRPC_CLIENT_ID_LEN);
+    }
+    memcpy(out->buf, bytes, len);
+    return NULL;
+}
+
+const char *hrpc_client_id_to_str(const struct hrpc_client_id *id,
+                           char *str, size_t str_len)
+{
+    snprintf(str, str_len, 
+             "%02x%02x%02x%02x-%02x%02x%02x%02x-%02x%02x%02x%02x-%02x%02x%02x%02x",
+             id->buf[0], id->buf[1], id->buf[2], id->buf[3],
+             id->buf[4], id->buf[5], id->buf[6], id->buf[7],
+             id->buf[8], id->buf[9], id->buf[10], id->buf[11],
+             id->buf[12], id->buf[13], id->buf[14], id->buf[15]);
+    return str;
+}
+
+int hrpc_client_id_compare(const struct hrpc_client_id *a,
+                            const struct hrpc_client_id *b)
+{
+    return memcmp(a->buf, b->buf, HRPC_CLIENT_ID_LEN);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 45 - 0
hadoop-native-core/rpc/client_id.h

@@ -0,0 +1,45 @@
+/**
+ * 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 HADOOP_CORE_RPC_CLIENT_ID
+#define HADOOP_CORE_RPC_CLIENT_ID
+
+#include <stddef.h> // for size_t
+#include <stdint.h> // for uint8_t
+
+#define HRPC_CLIENT_ID_LEN 16
+#define HRPC_CLIENT_ID_STR_LEN 36
+
+struct hrpc_client_id {
+    uint8_t buf[HRPC_CLIENT_ID_LEN];
+};
+
+void hrpc_client_id_generate_rand(struct hrpc_client_id *id);
+
+struct hadoop_err *hrpc_client_id_from_bytes(void *bytes, int len,
+                                             struct hrpc_client_id *out);
+
+const char *hrpc_client_id_to_str(const struct hrpc_client_id *id,
+                           char *str, size_t str_len);
+
+int hrpc_client_id_compare(const struct hrpc_client_id *a,
+                            const struct hrpc_client_id *b);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 669 - 0
hadoop-native-core/rpc/conn.c

@@ -0,0 +1,669 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "common/string.h"
+#include "protobuf/IpcConnectionContext.pb-c.h.s"
+#include "protobuf/ProtobufRpcEngine.pb-c.h.s"
+#include "protobuf/RpcHeader.pb-c.h.s"
+#include "rpc/call.h"
+#include "rpc/conn.h"
+#include "rpc/messenger.h"
+#include "rpc/reactor.h"
+#include "rpc/varint.h"
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <uv.h>
+
+#define conn_log_warn(conn, fmt, ...) \
+    fprintf(stderr, "WARN: conn %p (reactor %p): " fmt, \
+            conn, conn->reactor, __VA_ARGS__)
+#define conn_log_info(conn, fmt, ...) \
+    fprintf(stderr, "INFO: conn %p (reactor %p): " fmt, \
+            conn, conn->reactor, __VA_ARGS__)
+#define conn_log_debug(conn, fmt, ...) \
+    fprintf(stderr, "DEBUG: conn %p (reactor %p): " fmt, \
+            conn, conn->reactor, __VA_ARGS__)
+
+/**
+ * The maximum length that we'll allocate to hold a response from the server.
+ * This number includes the response header.
+ */
+#define MAX_READER_BODY_LEN (64 * 1024 * 1024)
+
+static const uint8_t FRAME[] = {
+    0x68, 0x72, 0x70, 0x63, // "hrpc"
+    0x09,                   // version
+    0x00,                   // service class
+    0x00                    // auth
+};
+
+#define FRAME_LEN sizeof(FRAME)
+
+static void conn_read_cb(uv_stream_t *stream, ssize_t nread,
+                         const uv_buf_t* buf);
+static void hrpc_conn_read_alloc(uv_handle_t *handle,
+                        size_t suggested_size, uv_buf_t *buf);
+
+static const char *conn_write_state_str(enum hrpc_conn_write_state state)
+{
+    switch (state) {
+    case HRPC_CONN_WRITE_CONNECTING:
+        return "HRPC_CONN_WRITE_CONNECTING";
+    case HRPC_CONN_WRITE_IPC_HEADER:
+        return "HRPC_CONN_WRITE_IPC_HEADER";
+    case HRPC_CONN_WRITE_PAYLOAD:
+        return "HRPC_CONN_WRITE_PAYLOAD";
+    case HRPC_CONN_WRITE_IDLE:
+        return "HRPC_CONN_WRITE_IDLE";
+    case HRPC_CONN_WRITE_CLOSED:
+        return "HRPC_CONN_WRITE_CLOSED";
+    default:
+        return "(unknown)";
+    }
+};
+
+static void free_write_bufs(struct hrpc_conn *conn)
+{
+    int i;
+    uv_buf_t *cur_writes = conn->writer.cur_writes;
+
+    for (i = 0; i < MAX_CUR_WRITES; i++) {
+        free(cur_writes[i].base);
+        cur_writes[i].base = NULL;
+        cur_writes[i].len = 0;
+    }
+}
+
+static struct hadoop_err *conn_setup_ipc_header(struct hrpc_conn *conn)
+{
+    struct hrpc_conn_writer *writer = &conn->writer;
+    IpcConnectionContextProto ipc_ctx = IPC_CONNECTION_CONTEXT_PROTO__INIT;
+    RpcRequestHeaderProto rpc_req_header = RPC_REQUEST_HEADER_PROTO__INIT;
+    int32_t cset_len, buf_len;
+    int32_t ipc_ctx_len, rpc_req_header_len, off = 0;
+    uint8_t *buf;
+    struct hadoop_err *err;
+
+    rpc_req_header.has_rpckind = 1;
+    rpc_req_header.rpckind = RPC_PROTOCOL_BUFFER;
+    rpc_req_header.has_rpcop = 1;
+    rpc_req_header.rpcop = RPC_FINAL_PACKET;
+    rpc_req_header.callid = -3; // ???
+    rpc_req_header.clientid.data = conn->client_id.buf;
+    rpc_req_header.clientid.len = HRPC_CLIENT_ID_LEN;
+    rpc_req_header.has_retrycount = 0;
+    rpc_req_header_len =
+        rpc_request_header_proto__get_packed_size(&rpc_req_header);
+
+    ipc_ctx.userinfo = NULL;
+    ipc_ctx.protocol = conn->protocol;
+    ipc_ctx_len = ipc_connection_context_proto__get_packed_size(&ipc_ctx);
+
+    cset_len = varint32_size(rpc_req_header_len) + rpc_req_header_len +
+        varint32_size(ipc_ctx_len) + ipc_ctx_len;
+    buf_len = FRAME_LEN + sizeof(uint32_t) + cset_len;
+    buf = malloc(buf_len);
+    if (!buf) {
+        err = hadoop_lerr_alloc(ENOMEM, "conn_setup_ipc_header: "
+                    "failed to allocate %d bytes", buf_len);
+        return err;
+    }
+    memcpy(buf, FRAME, FRAME_LEN);
+    off += FRAME_LEN;
+    be32_encode(cset_len, buf + off);
+    off += sizeof(uint32_t);
+    varint32_encode(rpc_req_header_len, buf, buf_len, &off);
+    rpc_request_header_proto__pack(&rpc_req_header, buf + off);
+    off += rpc_req_header_len;
+    varint32_encode(ipc_ctx_len, buf, buf_len, &off);
+    ipc_connection_context_proto__pack(&ipc_ctx, buf + off);
+    free_write_bufs(conn);
+    writer->cur_writes[0].base = (char*)buf;
+    writer->cur_writes[0].len = buf_len;
+    return NULL;
+}
+
+static struct hadoop_err *hrpc_conn_setup_payload(struct hrpc_conn *conn)
+{
+    struct hrpc_conn_writer *writer = &conn->writer;
+    struct hrpc_call *call = conn->call;
+    RpcRequestHeaderProto rpc_req_header = RPC_REQUEST_HEADER_PROTO__INIT;
+    int32_t rpc_req_header_len, header_buf_len, total_len, off = 0;
+    int64_t total_len64;
+    uint8_t *header_buf;
+    struct hadoop_err *err;
+
+    rpc_req_header.has_rpckind = 1;
+    rpc_req_header.rpckind = RPC_PROTOCOL_BUFFER;
+    rpc_req_header.has_rpcop = 1;
+    rpc_req_header.rpcop = RPC_FINAL_PACKET;
+    rpc_req_header.callid = 0; // ???
+    rpc_req_header.clientid.data = conn->client_id.buf;
+    rpc_req_header.clientid.len = HRPC_CLIENT_ID_LEN;
+    rpc_req_header.has_retrycount = 0;
+    rpc_req_header_len =
+        rpc_request_header_proto__get_packed_size(&rpc_req_header);
+
+    total_len64 = varint32_size(rpc_req_header_len);
+    total_len64 += rpc_req_header_len;
+    total_len64 += call->payload.len;
+    if (total_len64 > MAX_READER_BODY_LEN) {
+        err = hadoop_lerr_alloc(EINVAL, "hrpc_conn_setup_payload: "
+                "can't send a payload of length %"PRId64".  The maximum "
+                "payload length is %d", total_len64, MAX_READER_BODY_LEN);
+        return err;
+    }
+    total_len = (int32_t)total_len64;
+    header_buf_len = total_len - call->payload.len + sizeof(uint32_t);
+    header_buf = malloc(header_buf_len);
+    if (!header_buf) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_conn_setup_payload: "
+                    "failed to allocate %d bytes for header.", header_buf_len);
+        return err;
+    }
+    be32_encode(total_len, header_buf + off);
+    off += sizeof(uint32_t);
+    varint32_encode(rpc_req_header_len, header_buf, header_buf_len, &off);
+    rpc_request_header_proto__pack(&rpc_req_header, header_buf + off);
+    writer->cur_writes[0].base = (char*)header_buf;
+    writer->cur_writes[0].len = header_buf_len;
+    writer->cur_writes[1].base = call->payload.base;
+    writer->cur_writes[1].len = call->payload.len;
+    call->payload.base = NULL;
+    call->payload.len = 0;
+    return NULL;
+}
+
+static void conn_write_cb(uv_write_t* req, int status)
+{
+    struct hrpc_conn *conn = req->data;
+    struct hrpc_conn_writer *writer = &conn->writer;
+    struct hrpc_conn_reader *reader = &conn->reader;
+    struct hadoop_err *err;
+    int res;
+
+    if (status) {
+        err = hadoop_uverr_alloc(status,
+                "conn_write_cb got error"); 
+        hrpc_conn_destroy(conn, err);
+        return;
+    }
+    switch (writer->state) {
+    case HRPC_CONN_WRITE_IPC_HEADER:
+        free_write_bufs(conn);
+        writer->state = HRPC_CONN_WRITE_PAYLOAD;
+        err = hrpc_conn_setup_payload(conn);
+        if (err) {
+            hrpc_conn_destroy(conn, err);
+            return;
+        }
+        writer->write_req.data = conn;
+        res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream, 
+                     writer->cur_writes, 2, conn_write_cb);
+        if (res) {
+            err = hadoop_uverr_alloc(res,
+                    "failed to call uv_write on payload");
+            hrpc_conn_destroy(conn, err);
+            return;
+        }
+        break;
+    case HRPC_CONN_WRITE_PAYLOAD:
+        conn_log_debug(conn, "%s", "conn_write_cb: finished writing payload.  "
+                      "Now waiting for response.\n");
+        free_write_bufs(conn);
+        writer->state = HRPC_CONN_WRITE_IDLE;
+        reader->state = HRPC_CONN_READ_LEN;
+        conn->stream.data = conn;
+        res = uv_read_start((uv_stream_t*)&conn->stream, hrpc_conn_read_alloc,
+                          conn_read_cb);
+        if (res) {
+            err = hadoop_uverr_alloc(res, "uv_read_start failed");
+            hrpc_conn_destroy(conn, err);
+            return;
+        }
+        break;
+    default:
+        conn_log_warn(conn, "conn_write_cb: got an unexpected write "
+                      "event while in %s state.\n",
+                      conn_write_state_str(writer->state));
+        return;
+    }
+}
+
+static void conn_start_outbound(struct hrpc_conn *conn)
+{
+    struct hadoop_err *err = NULL;
+    struct hrpc_conn_writer *writer = &conn->writer;
+    int res;
+
+    writer->state = HRPC_CONN_WRITE_IPC_HEADER;
+    err = conn_setup_ipc_header(conn);
+    if (err) {
+        hrpc_conn_destroy(conn, err);
+        return;
+    }
+    writer->write_req.data = conn;
+    res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream,
+                 writer->cur_writes, 1, conn_write_cb);
+    if (res) {
+        err = hadoop_uverr_alloc(res,
+                "failed to call uv_write on ipc_header_buf");
+        hrpc_conn_destroy(conn, err);
+        return;
+    }
+}
+
+void hrpc_conn_start_outbound(struct hrpc_conn *conn, struct hrpc_call *call)
+{
+    conn->call = call;
+    conn_start_outbound(conn);
+}
+
+static void conn_connect_cb(uv_connect_t *req, int status)
+{
+    struct hrpc_conn *conn = req->data;
+    struct hadoop_err *err = NULL;
+    struct hrpc_conn_writer *writer = &conn->writer;
+
+    if (status) {
+        err = hadoop_uverr_alloc(status, "uv_tcp_connect failed");
+        hrpc_conn_destroy(conn, err);
+        return;
+    }
+    if (writer->state != HRPC_CONN_WRITE_CONNECTING) {
+        err = hadoop_lerr_alloc(EINVAL,
+                "got conn_connect_cb, but connection was not in "
+                "state HRPC_CONN_WRITE_CONNECTING.  state = %d",
+                writer->state);
+        hrpc_conn_destroy(conn, err);
+        return;
+    }
+    conn_start_outbound(conn);
+}
+
+struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
+                                    struct hrpc_call *call,
+                                    struct hrpc_conn **out)
+{
+    struct hadoop_err *err = NULL;
+    struct hrpc_conn *conn = NULL;
+    int res, tcp_init = 0;
+
+    conn = calloc(1, sizeof(struct hrpc_conn));
+    if (!conn) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_conn_create_outbound: OOM");
+        goto done;
+    }
+    conn->reactor = reactor;
+    conn->call = call;
+    conn->remote = call->remote;
+    conn->protocol = strdup(call->protocol);
+    if (!conn->protocol) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_conn_create_outbound: OOM");
+        goto done;
+    }
+    hrpc_client_id_generate_rand(&conn->client_id);
+    res = uv_tcp_init(&reactor->loop, &conn->stream);
+    if (res) {
+        err = hadoop_uverr_alloc(res,
+                "hrpc_conn_create_outbound: uv_tcp_init failed");
+        goto done;
+    }
+    tcp_init = 1;
+    conn->writer.state = HRPC_CONN_WRITE_CONNECTING;
+    conn->reader.state = HRPC_CONN_UNREADABLE;
+    conn->conn_req.data = conn;
+    res = uv_tcp_connect(&conn->conn_req, &conn->stream,
+            (struct sockaddr*)&conn->remote, conn_connect_cb);
+    if (res) {
+        err = hadoop_uverr_alloc(res,
+                "hrpc_conn_create_outbound: uv_tcp_connect failed");
+        goto done;
+    }
+
+done:
+    if (err) {
+        if (conn) {
+            free(conn->protocol);
+            if (tcp_init) {
+                uv_close((uv_handle_t*)&conn->stream, NULL);
+            }
+            free(conn);
+        }
+        return err;
+    }
+    *out = conn;
+    return NULL;
+}
+
+int hrpc_conn_compare(const struct hrpc_conn *a, const struct hrpc_conn *b)
+{
+    int proto_cmp, a_active, b_active;
+
+    // Big-endian versus little-endian doesn't matter here.
+    // We just want a consistent ordering on the same machine.
+    if (a->remote.sin_addr.s_addr < b->remote.sin_addr.s_addr)
+        return -1;
+    else if (a->remote.sin_addr.s_addr > b->remote.sin_addr.s_addr)
+        return 1;
+    else if (a->remote.sin_port < b->remote.sin_port)
+        return -1;
+    else if (a->remote.sin_port > b->remote.sin_port)
+        return 1;
+    // Compare protocol name.
+    proto_cmp = strcmp(a->protocol, b->protocol);
+    if (proto_cmp != 0)
+        return proto_cmp;
+    // Make the inactive connections sort before the active ones.
+    a_active = !!a->call;
+    b_active = !!b->call;
+    if (a_active < b_active) 
+        return -1;
+    else if (a_active > b_active) 
+        return 1;
+    // Compare pointer identity, so that no two distinct connections are
+    // ever identical.
+    else if (a < b)
+        return -1;
+    else if (a > b)
+        return 1;
+    return 0;
+}
+
+int hrpc_conn_usable(const struct hrpc_conn *conn,
+                      const struct sockaddr_in *addr, const char *protocol)
+{
+    if (conn->remote.sin_addr.s_addr != addr->sin_addr.s_addr)
+        return 0;
+    else if (conn->remote.sin_port != addr->sin_port)
+        return 0;
+    else if (strcmp(conn->protocol, protocol))
+        return 0;
+    return 1;
+}
+
+static void free_read_bufs(struct hrpc_conn *conn)
+{
+    free(conn->reader.body);
+    conn->reader.body = NULL;
+    conn->reader.body_len = 0;
+    conn->reader.off = 0;
+}
+
+static struct hadoop_err *conn_deliver_resp(struct hrpc_conn *conn,
+                              int32_t off, int32_t payload_len)
+{
+    struct hrpc_conn_reader *reader = &conn->reader;
+    struct hrpc_call *call = conn->call;
+    int64_t payload_end;
+    struct hrpc_response resp;
+
+    // Check if the server sent us a bogus payload_len value.
+    if (payload_len < 0) {
+        return hadoop_lerr_alloc(EIO, "conn_deliver_resp: "
+            "server's payload_len was %"PRId32", but negative payload "
+            "lengths are not valid.", payload_len);
+    }
+    payload_end = off;
+    payload_end += payload_len;
+    if (payload_end > reader->body_len) {
+        return hadoop_lerr_alloc(EIO, "conn_deliver_resp: "
+            "server's payload_len was %"PRId64", but there are only %d "
+            "bytes left in the body buffer.", payload_end, reader->body_len);
+    }
+    // Reset the connection's read state.  We'll hold on to the response buffer
+    // while making the callback.
+    resp.pb_base = (uint8_t*)(reader->body + off);
+    resp.pb_len = payload_len;
+    resp.base = reader->body;
+    reader->body = NULL;
+    free_read_bufs(conn);
+    conn->call = NULL;
+    conn->reader.state = HRPC_CONN_UNREADABLE;
+    // TODO: cache connections 
+    hrpc_conn_destroy(conn, NULL);
+    hrpc_call_deliver_resp(call, &resp);
+    return NULL;
+}
+
+static struct hadoop_err *conn_process_response(struct hrpc_conn *conn)
+{
+    struct hrpc_conn_reader *reader = &conn->reader;
+    RpcResponseHeaderProto *resp_header = NULL;
+    int32_t off = 0, resp_header_len, payload_len, rem;
+    struct hadoop_err *err = NULL;
+
+    if (varint32_decode(&resp_header_len, 
+                    (uint8_t*)reader->body, reader->body_len, &off)) {
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: response was "
+                "only %d bytes-- too short to read the rpc request header.",
+                reader->body_len);
+        goto done;
+    }
+    if (resp_header_len <= 0) {
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: server sent "
+                "invalid resp_header_len of %"PRId32, resp_header_len);
+        goto done;
+    }
+    rem = reader->body_len - off;
+    if (resp_header_len > rem) {
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: server sent "
+                "resp_header_len of %"PRId32", but there were only %"PRId32
+                " bytes left in the response.", resp_header_len, rem);
+        goto done;
+    }
+    resp_header = rpc_response_header_proto__unpack(NULL, resp_header_len,
+                                        (const uint8_t*)(reader->body + off));
+    if (!resp_header) {
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: failed to "
+                "parse RpcRequestHeaderProto.");
+        goto done;
+    }
+    off += resp_header_len;
+    if (resp_header->status != RPC_STATUS_PROTO__SUCCESS) {
+        // TODO: keep connection open if we got an ERROR rather than a FATAL.
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: error %s: %s",
+                  resp_header->exceptionclassname, resp_header->errormsg);
+        goto done;
+    }
+    if (varint32_decode(&payload_len, 
+                    (uint8_t*)reader->body, reader->body_len, &off)) {
+        err = hadoop_lerr_alloc(EIO, "conn_process_response: header was %d "
+                "bytes, and total length was %d-- too short to read the "
+                "payload.", resp_header_len, reader->body_len);
+        goto done;
+    }
+    err = conn_deliver_resp(conn, off, payload_len);
+done:
+    if (resp_header) {
+        rpc_response_header_proto__free_unpacked(resp_header, NULL);
+    }
+    return err;
+}
+
+static const char *conn_read_state_str(enum hrpc_conn_read_state state)
+{
+    switch (state) {
+    case HRPC_CONN_UNREADABLE:
+        return "HRPC_CONN_UNREADABLE";
+    case HRPC_CONN_READ_LEN:
+        return "HRPC_CONN_READ_LEN";
+    case HRPC_CONN_READ_BODY:
+        return "HRPC_CONN_READ_BODY";
+    case HRPC_CONN_READ_CLOSED:
+        return "HRPC_CONN_READ_CLOSED";
+    default:
+        return "(unknown)";
+    }
+};
+
+/**
+ * Return a read buffer to libuv.
+ *
+ * We don't do the actual allocation here, for two reasons.  The first is that
+ * we'd like to know how big a buffer to allocate first (by reading the first
+ * 4 bytes).  The second is that libuv doesn't really take kindly to
+ * failures here... returning a zero-length buffer triggers a crash.
+ * So we simply return previously allocated buffers here.
+ *
+ * @param handle            The TCP stream.
+ * @param suggested_size    The suggested size.
+ *
+ * @return                  The read buffer to use.
+ */
+static void hrpc_conn_read_alloc(uv_handle_t *handle,
+        size_t suggested_size __attribute__((unused)), uv_buf_t *buf)
+{
+    int32_t rem;
+    struct hrpc_conn *conn = handle->data;
+    struct hrpc_conn_reader *reader = &conn->reader;
+
+    switch (reader->state) {
+    case HRPC_CONN_READ_LEN:
+        buf->base = (char*)(reader->body_len_buf + reader->off);
+        buf->len = READLEN_BUF_LEN - reader->off;
+        return;
+    case HRPC_CONN_READ_BODY:
+        rem = reader->body_len - reader->off;
+        if (rem <= 0) {
+            conn_log_warn(conn, "hrpc_conn_read_alloc: we're in state "
+                    "HRPC_CONN_READ_BODY with reader->body_len = %"PRId32", but "
+                    "reader->off = %"PRId32"\n", reader->body_len, reader->off);
+            buf->base = NULL;
+            buf->len = 0;
+            return;
+        }
+        buf->base = (char*)(reader->body + reader->off);
+        buf->len = rem;
+        return;
+    default:
+        conn_log_warn(conn, "hrpc_conn_read_alloc: we're in state "
+                "%s, but we're being asked to allocate "
+                "a read buffer.\n", conn_read_state_str(reader->state));
+        buf->base = NULL;
+        buf->len = 0;
+        return;
+    }
+}
+
+/**
+ * The read callback for this connection.
+ */
+static void conn_read_cb(uv_stream_t *stream, ssize_t nread,
+                         const uv_buf_t* buf __attribute__((unused)))
+{
+    struct hrpc_conn *conn = stream->data;
+    struct hrpc_conn_reader *reader = &conn->reader;
+    struct hadoop_err *err;
+
+    if (nread < 0) {
+        hrpc_conn_destroy(conn,
+            hadoop_uverr_alloc(-nread, "conn_read_cb error"));
+        return;
+    }
+    if (nread == 0) {
+        // Nothing to do.
+        return;
+    }
+    switch (reader->state) {
+    case HRPC_CONN_READ_LEN:
+        reader->off += nread;
+        if (reader->off < READLEN_BUF_LEN) {
+            conn_log_debug(conn, "conn_read_cb: got partial read of "
+                           "body_len.  reader->off = %"PRId32"\n",
+                           reader->off);
+            return;
+        }
+        reader->body_len = be32_decode(reader->body_len_buf);
+        if ((reader->body_len <= 0) ||
+                (reader->body_len > MAX_READER_BODY_LEN)) {
+            hrpc_conn_destroy(conn, hadoop_lerr_alloc(EIO, 
+                "conn_read_cb: got an invalid body length of %"PRId32"\n", 
+                reader->body_len));
+            return;
+        }
+        conn_log_debug(conn, "conn_read_cb: got body length of "
+                       "%"PRId32".  Transitioning to HRPC_CONN_READ_BODY.\n",
+                       reader->body_len);
+        reader->off = 0;
+        reader->state = HRPC_CONN_READ_BODY;
+        reader->body = malloc(reader->body_len);
+        if (!reader->body) {
+            hrpc_conn_destroy(conn, hadoop_lerr_alloc(ENOMEM,
+                    "hrpc_conn_read_alloc: failed to allocate "
+                    "%"PRId32" bytes.\n", reader->body_len));
+        }
+        break;
+    case HRPC_CONN_READ_BODY:
+        reader->off += nread;
+        if (reader->off < reader->body_len) {
+            conn_log_debug(conn, "conn_read_cb: got partial read of "
+                           "body.  reader->off = %"PRId32" out of %"PRId32"\n",
+                           reader->off, reader->body_len);
+            return;
+        }
+        err = conn_process_response(conn);
+        free_read_bufs(conn);
+        if (err) {
+            hrpc_conn_destroy(conn, err);
+            return;
+        }
+        reader->state = HRPC_CONN_UNREADABLE;
+        break;
+
+    default:
+        conn_log_warn(conn, "conn_read_cb: got an unexpected read "
+                      "event while in %s state.\n",
+                      conn_read_state_str(reader->state));
+        return;
+    }
+}
+
+static void conn_free(uv_handle_t* handle)
+{
+    struct hrpc_conn *conn = handle->data;
+    free(conn);
+}
+
+void hrpc_conn_destroy(struct hrpc_conn *conn, struct hadoop_err *err)
+{
+    reactor_remove_conn(conn->reactor, conn);
+    if (conn->call) {
+        err = err ? err : hadoop_lerr_alloc(EFAULT, "hrpc_conn_destroy: "
+                "internal error: shutting down connection while it "
+                "still has a call in progress.");
+        conn_log_warn(conn, "hrpc_conn_destroy: %s\n", hadoop_err_msg(err));
+        hrpc_call_deliver_err(conn->call, err);
+        conn->call = NULL;
+    } else if (err) {
+        conn_log_warn(conn, "hrpc_conn_destroy: %s\n", hadoop_err_msg(err));
+        hadoop_err_free(err);
+    }
+    free_read_bufs(conn);
+    conn->reader.state = HRPC_CONN_READ_CLOSED;
+    free_write_bufs(conn);
+    conn->writer.state = HRPC_CONN_WRITE_CLOSED;
+    free(conn->protocol);
+    uv_close((uv_handle_t*)&conn->stream, conn_free);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 218 - 0
hadoop-native-core/rpc/conn.h

@@ -0,0 +1,218 @@
+/**
+ * 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 HADOOP_CORE_RPC_CONNECTION_H
+#define HADOOP_CORE_RPC_CONNECTION_H
+
+#include "common/tree.h"
+#include "rpc/client_id.h"
+
+#include <stdint.h>
+#include <uv.h>
+
+struct hrpc_call;
+
+#define MAX_CUR_WRITES 2
+#define READLEN_BUF_LEN 4
+
+enum hrpc_conn_write_state {
+    /**
+     * The state when we're still calling connect(2).
+     */
+    HRPC_CONN_WRITE_CONNECTING,
+
+    /**
+     * The write state where we're sending the frame, IPC header, etc.
+     * TODO: implement SASL and its associated states.
+     */
+    HRPC_CONN_WRITE_IPC_HEADER,
+
+    /**
+     * The write state where we're writing the RpcRequestHeaderProto and
+     * request payload.
+     */
+    HRPC_CONN_WRITE_PAYLOAD,
+
+    /**
+     * The write state where we have nothing to write.
+     */
+    HRPC_CONN_WRITE_IDLE,
+
+    /**
+     * Closed state.
+     */
+    HRPC_CONN_WRITE_CLOSED,
+};
+
+enum hrpc_conn_read_state {
+    /**
+     * The read state in which we don't expect read callbacks.
+     * Generally, we're only in this state when the connection itself has not
+     * been established.
+     */
+    HRPC_CONN_UNREADABLE = 0,
+
+    /**
+     * The read state in which we're reading the 4-byte length prefix.
+     */
+    HRPC_CONN_READ_LEN,
+
+    /**
+     * The read state in which we're reading the message body.
+     */
+    HRPC_CONN_READ_BODY,
+
+    /**
+     * Closed state.
+     */
+    HRPC_CONN_READ_CLOSED,
+};
+
+struct hrpc_conn_reader {
+    enum hrpc_conn_read_state state; //!< Current read state.
+    uint8_t body_len_buf[READLEN_BUF_LEN];//!< The buffer for body length.
+    int32_t body_len;            //!< Body length to read.
+    int32_t off;                 //!< Current offset.
+    char *body;                  //!< malloc'ed message body we're reading.
+};
+
+struct hrpc_conn_writer {
+    enum hrpc_conn_write_state state;//!< Current write state.
+    uv_write_t write_req;
+    uv_buf_t cur_writes[MAX_CUR_WRITES];
+};
+
+/**
+ * A Hadoop connection.
+ *
+ * This object manages the TCP connection with the remote.
+ * Note that we can read and write from the remote simultaneously;
+ * that's why write_state and read_state are separate.
+ */
+struct hrpc_conn {
+    RB_ENTRY(hrpc_conn) entry;
+
+    /**
+     * The reactor that owns this connection.
+     */
+    struct hrpc_reactor *reactor;
+
+    /**
+     * The call we're handling.
+     */
+    struct hrpc_call *call;
+
+    /**
+     * The remote address we're connected to.
+     */
+    struct sockaddr_in remote;         
+
+    /**
+     * This connection's TCP stream.
+     */
+    uv_tcp_t stream;
+
+    /**
+     * The Hadoop protocol we're talking to, such as
+     * org.apache.hadoop.hdfs.protocol.ClientProtocol.  Malloc'ed.
+     */
+    char *protocol;
+
+    /**
+     * The client ID we used when establishing the connection.
+     */
+    struct hrpc_client_id client_id;  
+
+    /**
+     * The pending connection request, if one is pending.
+     */
+    uv_connect_t conn_req;
+
+    struct hrpc_conn_writer writer;
+    struct hrpc_conn_reader reader;
+};
+
+/**
+ * Create a new hrpc_conn to the given remote, using bind, connect, etc.
+ *
+ * @param reactor       The reactor that the connection will be associated
+ *                          with.
+ * @param call          The call to make.  The connection will take ownership
+ *                          of this call.  If the connection fails, the call
+ *                          will be given a failure callback.
+ * @param out           (out param) On success, the new connection.
+ *
+ * @return              NULL on success; the error otherwise.
+ */
+struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
+                                    struct hrpc_call *call,
+                                    struct hrpc_conn **out);
+
+/**
+ * Start an outbound call on this connection.
+ *
+ * @param conn          The connection.
+ * @param call          The call.
+ */
+void hrpc_conn_start_outbound(struct hrpc_conn *conn, struct hrpc_call *call);
+
+/**
+ * Compare two hadoop connection objects.
+ *
+ * Comparison is done lexicographically by:
+ *    - IP address.
+ *    - Port.
+ *    - Whether the connection is in use.
+ *    - Memory Address.
+ *
+ * By doing the comparison this way, we make it easy to find the first idle
+ * connection to a given remote, or quickly determine that there is not one.
+ * We also allow multiple connections to the same address.
+ *
+ * @param a             The first connection object.
+ * @param b             The second connection object.
+ *
+ * @return              negative if a < b; positive if a > b; 0 otherwise.
+ */
+int hrpc_conn_compare(const struct hrpc_conn *a,
+                      const struct hrpc_conn *b);
+
+/**
+ * Determine if a connection is usable for a given address and protocol.
+ *
+ * @param conn          The connection.
+ * @param addr          The address.
+ * @param protocol      The protocol.
+ *
+ * @return              1 if the connection is usable; 0 if not.
+ */
+int hrpc_conn_usable(const struct hrpc_conn *conn,
+                      const struct sockaddr_in *addr, const char *protocol);
+
+/**
+ * Destroy a connection.
+ *
+ * @param conn          The connection to destroy.
+ * @param err           The error.  This will be delivered to any callback that
+ *                          the connection owns.
+ */
+void hrpc_conn_destroy(struct hrpc_conn *conn, struct hadoop_err *err);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 116 - 0
hadoop-native-core/rpc/messenger.c

@@ -0,0 +1,116 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "rpc/conn.h"
+#include "rpc/messenger.h"
+#include "rpc/reactor.h"
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <uv.h>
+
+#define msgr_log_warn(msgr, fmt, ...) \
+    fprintf(stderr, "WARN: msgr %p: " fmt, msgr, __VA_ARGS__)
+#define msgr_log_info(msgr, fmt, ...) \
+    fprintf(stderr, "INFO: msgr %p: " fmt, msgr, __VA_ARGS__)
+#define msgr_log_debug(msgr, fmt, ...) \
+    fprintf(stderr, "DBUG: msgr %p: " fmt, msgr, __VA_ARGS__)
+
+struct hrpc_messenger_builder {
+};
+
+/**
+ * The Hadoop Messenger.
+ *
+ * The messenger owns all the reactor threads, and is the main entry point into
+ * the RPC system.
+ */
+struct hrpc_messenger {
+    /**
+     * The reactor thread which makes the actual network calls.
+     *
+     * TODO: support multiple reactor threads.
+     */
+    struct hrpc_reactor *reactor;
+};
+
+struct hrpc_messenger_builder *hrpc_messenger_builder_alloc(void)
+{
+    struct hrpc_messenger_builder *bld;
+
+    bld = calloc(1, sizeof(struct hrpc_messenger_builder));
+    if (!bld)
+        return NULL;
+    return bld;
+}
+
+void hrpc_messenger_builder_free(struct hrpc_messenger_builder *bld)
+{
+    if (!bld)
+        return;
+    free(bld);
+}
+
+struct hadoop_err *hrpc_messenger_create(
+        struct hrpc_messenger_builder *bld, struct hrpc_messenger **out)
+{
+    struct hrpc_messenger *msgr = NULL;
+    struct hadoop_err *err = NULL;
+
+    free(bld);
+    msgr = calloc(1, sizeof(struct hrpc_messenger));
+    if (!msgr) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_messenger_create: OOM");
+        goto error;
+    }
+    err = hrpc_reactor_create(&msgr->reactor);
+    if (err) {
+        goto error_free_msgr;
+    }
+    msgr_log_info(msgr, "created messenger %p\n", msgr);
+    *out = msgr;
+    return NULL;
+
+error_free_msgr:
+    free(msgr);
+error:
+    return err;
+}
+
+void hrpc_messenger_start_outbound(struct hrpc_messenger *msgr,
+                                   struct hrpc_call *call)
+{
+    hrpc_reactor_start_outbound(msgr->reactor, call);
+}
+
+void hrpc_messenger_shutdown(struct hrpc_messenger *msgr)
+{
+    msgr_log_debug(msgr, "%s", "hrpc_messenger_shutdown\n");
+    hrpc_reactor_shutdown(msgr->reactor);
+}
+
+void hrpc_messenger_free(struct hrpc_messenger *msgr)
+{
+    msgr_log_debug(msgr, "%s", "hrpc_messenger_free\n");
+    hrpc_reactor_free(msgr->reactor);
+    free(msgr);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 94 - 0
hadoop-native-core/rpc/messenger.h

@@ -0,0 +1,94 @@
+/**
+ * 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 HADOOP_CORE_RPC_MSGR_H
+#define HADOOP_CORE_RPC_MSGR_H
+
+struct hrpc_call;
+struct hrpc_messenger;
+struct hrpc_messenger_builder;
+
+#include <stdint.h> // for int32_t
+
+/**
+ * Allocate a Hadoop messenger builder.
+ *
+ * @return              A Hadoop messenger builder, or NULL on OOM.
+ */
+struct hrpc_messenger_builder *hrpc_messenger_builder_alloc(void);
+
+/**
+ * Set the timeout for the Hadoop messenger.
+ *
+ * @param bld           The messenger builder.
+ * @param timeout_ms    The timeout in milliseconds
+ */
+void hrpc_messenger_builder_set_timeout_ms(struct hrpc_messenger_builder *bld,
+                                           int32_t timeout_ms);
+
+/**
+ * Free a Hadoop messenger builder.
+ *
+ * @param bld           The Hadoop proxy builder to free.
+ */
+void hrpc_messenger_builder_free(struct hrpc_messenger_builder *bld);
+
+/**
+ * Create a Hadoop messenger.
+ *
+ * @param bld           The Hadoop messenger builder to use.
+ *                        You must still free the builder even after calling
+ *                        this function.
+ * @param out       (out param) On success, the Hadoop messenger.
+ *
+ * @return          On success, NULL.  On error, the error.
+ */
+struct hadoop_err *hrpc_messenger_create(struct hrpc_messenger_builder *bld,
+                                         struct hrpc_messenger **out);
+
+/**
+ * Start an outbound call.
+ *
+ * @param msgr          The messenger.
+ * @param call          The call.
+ */
+void hrpc_messenger_start_outbound(struct hrpc_messenger *msgr,
+                                   struct hrpc_call *call);
+
+/**
+ * Shut down the messenger.
+ *
+ * After this function is called, the messenger will stop accepting new calls.
+ * We will deliver ESHUTDOWN to existing calls.
+ *
+ * @param msgr          The messenger.
+ */
+void hrpc_messenger_shutdown(struct hrpc_messenger *msgr);
+
+/**
+ * De-allocate the memory and other resources associated with this messenger.
+ *
+ * After this function is called, the messenger pointer will be invalid.
+ *
+ * @param msgr          The messenger.
+ */
+void hrpc_messenger_free(struct hrpc_messenger *msgr);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 456 - 0
hadoop-native-core/rpc/protoc-gen-hrpc.cc

@@ -0,0 +1,456 @@
+/**
+ * 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 <ctype.h>
+#include <google/protobuf/compiler/code_generator.h>
+#include <google/protobuf/compiler/plugin.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/io/printer.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+#include <google/protobuf/stubs/common.h>
+#include <tr1/memory>
+
+#include <iostream>
+#include <map>
+#include <stdio.h>
+#include <string>
+
+#define PROTO_EXTENSION ".proto"
+
+#define APACHE_HEADER \
+"/**\n" \
+" * Licensed to the Apache Software Foundation (ASF) under one\n" \
+" * or more contributor license agreements.  See the NOTICE file\n" \
+" * distributed with this work for additional information\n" \
+" * regarding copyright ownership.  The ASF licenses this file\n" \
+" * to you under the Apache License, Version 2.0 (the\n" \
+" * \"License\"); you may not use this file except in compliance\n" \
+" * with the License.  You may obtain a copy of the License at\n" \
+" *\n" \
+" *     http://www.apache.org/licenses/LICENSE-2.0\n" \
+" *\n" \
+" * Unless required by applicable law or agreed to in writing, software\n" \
+" * distributed under the License is distributed on an \"AS IS\" BASIS,\n" \
+" * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" \
+" * See the License for the specific language governing permissions and\n" \
+" * limitations under the License.\n" \
+" */\n"
+
+using google::protobuf::FileDescriptor;
+using google::protobuf::MethodDescriptor;
+using google::protobuf::ServiceDescriptor;
+using google::protobuf::compiler::GeneratorContext;
+using google::protobuf::io::Printer;
+using std::map;
+using std::string;
+using std::tr1::shared_ptr;
+
+typedef map<string, string> string_map_t;
+
+static string camel_case_to_uscore(const string &in)
+{
+    string out;
+    bool prev_lower = false;
+
+    for (size_t i = 0; i < in.size(); i++) {
+        char c = in[i];
+        if (isupper(c)) {
+            if (prev_lower) {
+                out += "_";
+            }
+            prev_lower = false;
+        } else if (islower(c) || isdigit(c)) {
+            prev_lower = true;
+        } else {
+            prev_lower = false;
+        }
+        out += tolower(c);
+    }
+    return out;
+}
+
+static bool try_strip_suffix(const string &str, const char *suffix,
+                                 string *out)
+{
+    size_t suffix_len = strlen(suffix);
+
+    if (str.size() < suffix_len) {
+        return false;
+    }
+    *out = str.substr(0, str.size() - suffix_len);
+    return true;
+}
+
+static void get_base_name(const string &path, string *base)
+{
+    size_t last_slash = path.find_last_of("/");
+    if (last_slash != string::npos) {
+        *base = path.substr(last_slash + 1);
+    } else {
+        *base = path;
+    }
+}
+
+static string set_path_substitutions(const FileDescriptor *file,
+                                     string_map_t *map)
+{
+    string path = file->name();
+    (*map)["path"] = path;
+    // Initialize path_
+    // If path = /foo/bar/baz_stuff.proto, path_ = /foo/bar/baz_stuff
+    string path_without_extension;
+    if (!try_strip_suffix(path, PROTO_EXTENSION, &path_without_extension)) {
+        return string("file name " + path + " did not end in " +
+                      PROTO_EXTENSION);
+    }
+    (*map)["path_without_extension"] = path_without_extension;
+
+    // If path = /foo/bar/baz_stuff.proto, base_ = baz_stuff
+    string base;
+    get_base_name(path_without_extension, &base);
+    (*map)["path_base"] = base;
+    (*map)["function_prefix"] = base;
+    return "";
+}
+
+static string shorten_service_prefix(const string &prefix)
+{
+    if (prefix == "ClientNamenodeProtocol") {
+        return "cnn";
+    } else if (prefix == "ClientDatanodeProtocolService") {
+        return "cdn";
+    } else if (prefix == "NamenodeProtocolService") {
+        return "nnp";
+    } else if (prefix == "DatanodeProtocolService") {
+        return "dn";
+    } else {
+        return prefix;
+    }
+}
+
+static void set_service_substitutions(const ServiceDescriptor *service,
+                                     string_map_t *map)
+{
+    // Service prefix.
+    // example: cnn
+    (*map)["service_prefix"] = shorten_service_prefix(service->name());
+}
+ 
+/**
+ * Process a dot-separated type name into a protobuf-c type name.
+ *
+ * @param input             The input type name.
+ *
+ * @return                  The protobuf-c type name.
+ */
+static string get_pbc_type_name(string input)
+{
+    char *word, *ptr = NULL;
+    string output, prefix;
+    char line[input.size() + 1];
+    strcpy(line, input.c_str());
+
+    for (word = strtok_r(line, ".", &ptr); word; 
+             word = strtok_r(NULL, ".", &ptr)) {
+        //fprintf(stderr, "word = %s\n", word);
+        if (!isupper(word[0])) {
+            word[0] = toupper(word[0]);
+        }
+        output += prefix;
+        prefix = "__";
+        output += word;
+    }
+    return output;
+}
+
+static string replace(string input, char target,
+                      const char *replacement)
+{
+    string output;
+
+    for (size_t i = 0; i < input.size(); i++) {
+        if (input[i] == target) {
+            output += replacement;
+        } else {
+            output += input[i];
+        }
+    }
+    return output;
+}
+
+static void set_method_substitutions(const MethodDescriptor *method,
+                                     string_map_t *map)
+{
+    // Request type, in camelcase.
+    // example: Hadoop__Hdfs__SetReplicationRequestProto 
+    (*map)["req_ty_camel"] =
+        get_pbc_type_name(method->input_type()->full_name());
+
+    // Request type, in underscore-separated lowercase.
+    // example: hadoop__hdfs__set_replication_request_proto
+    (*map)["req_ty_uscore"] = camel_case_to_uscore((*map)["req_ty_camel"]);
+
+    // Response type, in camelcase.
+    // example: Hadoop__Hdfs__SetReplicationResponseProto 
+    (*map)["resp_ty_camel"] =
+        get_pbc_type_name(method->output_type()->full_name());
+
+    // Response type, in underscore-separated lowercase.
+    // example: hadoop__hdfs__set_replication_response_proto
+    (*map)["resp_ty_uscore"] = camel_case_to_uscore((*map)["resp_ty_camel"]);
+
+    // RPC name, in camelcase.
+    // example: setReplication
+    (*map)["rpc_camel"] = method->name();
+
+    // RPC name, in underscore-separated lowercase.
+    // example: setReplication
+    (*map)["rpc_uscore"] = camel_case_to_uscore((*map)["rpc_camel"]);
+
+    // sync stub function name.
+    // example: cnn_set_replication
+    (*map)["sync_call"] =
+        (*map)["service_prefix"] + "_" + (*map)["rpc_uscore"];
+
+    // async stub function name.
+    // example: cnn_async_set_replication
+    (*map)["async_call"] =
+        (*map)["service_prefix"] + "_async_" + (*map)["rpc_uscore"];
+
+    // async callback adaptor function name.
+    // example: cnn_async_adaptor_set_replication
+    (*map)["async_adaptor"] =
+        (*map)["service_prefix"] + "_async_adaptor_" + (*map)["rpc_uscore"];
+}
+
+class HrpcCodeGenerator
+    : public ::google::protobuf::compiler::CodeGenerator
+{
+public:
+    HrpcCodeGenerator()
+    {
+    }
+
+    ~HrpcCodeGenerator()
+    {
+    }
+
+    bool Generate(const google::protobuf::FileDescriptor *file,
+        const string &, GeneratorContext *gen_context,
+        string *error) const
+    {
+        string_map_t path_map;
+        string ret = set_path_substitutions(file, &path_map);
+        if (!ret.empty()) {
+            *error = ret;
+            return false;
+        }
+        generate_call_header(gen_context, &path_map, file);
+        generate_call_body(gen_context, &path_map, file);
+        return true;
+    }
+
+private:
+    void generate_call_header(GeneratorContext *gen_context,
+                              string_map_t *path_map,
+                              const FileDescriptor *file) const
+    {
+        shared_ptr<google::protobuf::io::ZeroCopyOutputStream> output(
+            gen_context->Open((*path_map)["path_without_extension"] +
+                              ".call.h"));
+        Printer printer(output.get(), '$');
+        printer.Print(APACHE_HEADER);
+        printer.Print(*path_map,
+"\n"
+"// This header file was auto-generated from $path$\n"
+"\n"
+"#ifndef HADOOP_NATIVE_CORE_$path_base$_CALL_H\n"
+"#define HADOOP_NATIVE_CORE_$path_base$_CALL_H\n"
+"\n"
+"#include \"protobuf/$path_base$.pb-c.h\"\n"
+"#include \"protobuf/$path_base$.pb-c.h.s\"\n"
+"\n"
+"struct hadoop_err;\n"
+"struct hrpc_proxy;\n"
+"\n");
+        for (int service_idx = 0; service_idx < file->service_count();
+                    ++service_idx) {
+            string_map_t service_map = *path_map;
+            const ServiceDescriptor *service = file->service(service_idx);
+            set_service_substitutions(service, &service_map);
+            for (int method_idx = 0; method_idx < service->method_count();
+                    ++method_idx) {
+                const MethodDescriptor *method = service->method(method_idx);
+                string_map_t map = service_map;
+                set_method_substitutions(method, &map);
+                printer.Print(map,
+"struct hadoop_err *$sync_call$(struct hrpc_proxy *proxy,\n"
+"               const $req_ty_camel$ *req,\n"
+"               $resp_ty_camel$ **resp);\n"
+"\n"
+"void $async_call$(struct hrpc_proxy *proxy,\n"
+"    const $req_ty_camel$ *req,\n"
+"    void (*cb)($resp_ty_camel$ *,\n"
+"        struct hadoop_err *, void *cb_data),\n"
+"    void *cb_data);\n"
+"\n");
+            }
+        }
+        printer.Print("#endif\n");
+    }
+
+    void generate_call_body(GeneratorContext *gen_context,
+                            string_map_t *path_map,
+                            const FileDescriptor *file) const
+    {
+        shared_ptr<google::protobuf::io::ZeroCopyOutputStream> output(
+            gen_context->Open((*path_map)["path_without_extension"] +
+                              ".call.c"));
+        Printer printer(output.get(), '$');
+        printer.Print(APACHE_HEADER);
+        printer.Print(*path_map, "\n"
+"#include \"common/hadoop_err.h\"\n"
+"#include \"protobuf/$path_base$.call.h\"\n"
+"#include \"rpc/messenger.h\"\n"
+"#include \"rpc/proxy.h\"\n");
+        printer.Print("\n"
+"#include <errno.h>\n"
+"#include <netinet/in.h>\n"
+"#include <stdio.h>\n"
+"#include <stdlib.h>\n"
+"#include <string.h>\n"
+"#include <uv.h>\n"
+"\n");
+        for (int service_idx = 0; service_idx < file->service_count();
+                    ++service_idx) {
+            string_map_t service_map = *path_map;
+            const ServiceDescriptor *service = file->service(service_idx);
+            set_service_substitutions(service, &service_map);
+            for (int method_idx = 0; method_idx < service->method_count();
+                    ++method_idx) {
+                const MethodDescriptor *method = service->method(method_idx);
+                string_map_t map = service_map;
+                set_method_substitutions(method, &map);
+                printer.Print(map,
+"struct hadoop_err *$sync_call$(struct hrpc_proxy *proxy,\n"
+"    const $req_ty_camel$ *req,\n"
+"    $resp_ty_camel$ **out)\n"
+"{\n"
+"    struct hadoop_err *err;\n"
+"    struct hrpc_sync_ctx *ctx;\n"
+"    $resp_ty_camel$ *resp;\n"
+"\n"
+"    err = hrpc_proxy_activate(proxy);\n"
+"    if (err) {\n"
+"        return err;\n"
+"    }\n"
+"    ctx = hrpc_proxy_alloc_sync_ctx(proxy);\n"
+"    if (!ctx) {\n"
+"        hrpc_proxy_deactivate(proxy);\n"
+"        return hadoop_lerr_alloc(ENOMEM, \"$sync_call$: \"\n"
+"            \"failed to allocate sync_ctx\");\n"
+"    }\n"
+"    hrpc_proxy_start(proxy, \"$rpc_camel$\", req,\n"
+"        $req_ty_uscore$__get_packed_size(req),\n"
+"        (hrpc_pack_cb_t)$req_ty_uscore$__pack,\n"
+"        hrpc_proxy_sync_cb, ctx);\n"
+"    if (ctx->err) {\n"
+"        hrpc_free_sync_ctx(ctx);\n"
+"        return ctx->err;\n"
+"    }\n"
+"    resp = $resp_ty_uscore$__unpack(NULL, ctx->resp.pb_len,\n"
+"                                                  ctx->resp.pb_base);\n"
+"    hrpc_free_sync_ctx(ctx);\n"
+"    if (!resp) {\n"
+"        return hadoop_lerr_alloc(EINVAL,\n"
+"           \"$sync_call$: failed to parse response from server\");\n"
+"    }\n"
+"    *out = resp;\n"
+"    return NULL;\n"
+"}\n");
+                printer.Print(map,
+"struct $async_call$_cb_data {\n"
+"    void (*cb)($resp_ty_camel$ *,\n"
+"        struct hadoop_err *, void *);\n"
+"    void *cb_data;\n"
+"};\n"
+"\n"
+"void $async_adaptor$(struct hrpc_response *resp,\n"
+"                        struct hadoop_err *err, void *cb_data)\n"
+"{\n"
+"    struct $async_call$_cb_data *wrapped = cb_data;\n"
+"    $resp_ty_camel$ *msg;\n"
+"\n"
+"    if (err) {\n"
+"        wrapped->cb(NULL, err, wrapped->cb_data);\n"
+"        return;\n"
+"    }\n"
+"    msg = $resp_ty_uscore$__unpack(NULL, resp->pb_len,\n"
+"                                                  resp->pb_base);\n"
+"    free(resp->base);\n"
+"    if (!msg) {\n"
+"        wrapped->cb(NULL, hadoop_lerr_alloc(EIO,\n"
+"            \"$async_adaptor$: \"\n"
+"            \"failed to parse response from server.\"), wrapped->cb_data);\n"
+"        return;\n"
+"    }\n"
+"    wrapped->cb(msg, NULL, wrapped->cb_data);\n"
+"}\n");
+                printer.Print(map,
+"void $async_call$(struct hrpc_proxy *proxy,\n"
+"    const $req_ty_camel$ *req,\n"
+"    void (*cb)($resp_ty_camel$ *,\n"
+"        struct hadoop_err *, void *),\n"
+"    void *cb_data)\n"
+"{\n"
+"    struct $async_call$_cb_data *wrapped;\n"
+"    struct hadoop_err *err;\n"
+"\n"
+"    err = hrpc_proxy_activate(proxy);\n"
+"    if (err) {\n"
+"        cb(NULL, err, cb_data);\n"
+"        return;\n"
+"    }\n"
+"    wrapped = hrpc_proxy_alloc_userdata(proxy, sizeof(*wrapped));\n"
+"    if (!wrapped) {\n"
+"        hrpc_proxy_deactivate(proxy);\n"
+"        cb(NULL, hadoop_lerr_alloc(ENOMEM, \"$async_call$: failed \"\n"
+"                                 \"to allocate sync_ctx\"), cb_data);\n"
+"        return;\n"
+"    }\n"
+"    wrapped->cb = cb;\n"
+"    wrapped->cb_data = cb_data;\n"
+"    hrpc_proxy_start(proxy, \"$rpc_camel$\", req, \n"
+"        $req_ty_uscore$__get_packed_size(req),\n"
+"        (hrpc_pack_cb_t)$req_ty_uscore$__pack,\n"
+"        $async_adaptor$, wrapped);\n"
+"}\n"
+"\n");
+            }
+        }
+        printer.Print("// vim: ts=4:sw=4:tw=79:et\n");
+    }
+};
+
+int main(int argc, char *argv[])
+{
+  HrpcCodeGenerator generator;
+  return google::protobuf::compiler::PluginMain(argc, argv, &generator);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 287 - 0
hadoop-native-core/rpc/proxy.c

@@ -0,0 +1,287 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "protobuf/ProtobufRpcEngine.pb-c.h.s"
+#include "rpc/call.h"
+#include "rpc/messenger.h"
+#include "rpc/proxy.h"
+#include "rpc/varint.h"
+
+#include <errno.h>
+#include <netinet/in.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <uv.h>
+
+#define proxy_log_warn(proxy, fmt, ...) \
+    fprintf(stderr, "WARN: proxy 0x%p: " fmt, proxy, __VA_ARGS__)
+#define proxy_log_info(proxy, fmt, ...) \
+    fprintf(stderr, "INFO: proxy 0x%p: " fmt, proxy, __VA_ARGS__)
+#define proxy_log_debug(proxy, fmt, ...) \
+    fprintf(stderr, "DEBUG: proxy 0x%p: " fmt, proxy, __VA_ARGS__)
+
+/**
+ * The maximum length that we'll allocate to hold a request to the server.
+ * This number includes the RequestHeader, but not the RpcRequestHeader.
+ */
+#define MAX_SEND_LEN (63 * 1024 * 1024)
+
+struct hrpc_proxy {
+    /**
+     * The messenger that this proxy is associated with.
+     */
+    struct hrpc_messenger *msgr;
+
+    /**
+     * Dynamically allocated string describing the protocol this proxy speaks. 
+     */
+    char *protocol;
+
+    /**
+     * The current call.
+     */
+    struct hrpc_call call;
+
+    /**
+     * A memory area which can be used by the current call.
+     *
+     * This will be null if userdata_len is 0.
+     */
+    uint8_t *userdata;
+
+    /**
+     * Length of userdata.
+     */
+    size_t userdata_len;
+};
+
+struct hrpc_proxy_builder {
+    struct hrpc_proxy *proxy;
+};
+
+static const char OOM_ERROR[] = "OOM";
+
+struct hrpc_proxy_builder *hrpc_proxy_builder_alloc(
+            struct hrpc_messenger *msgr)
+{
+    struct hrpc_proxy_builder *bld;
+
+    bld = calloc(1, sizeof(struct hrpc_proxy_builder));
+    if (!bld)
+        return NULL;
+    bld->proxy = calloc(1, sizeof(struct hrpc_proxy));
+    if (!bld->proxy) {
+        free(bld);
+        return NULL;
+    }
+    bld->proxy->msgr = msgr;
+    bld->proxy->call.remote.sin_addr.s_addr = INADDR_ANY;
+
+    return bld;
+}
+
+void hrpc_proxy_builder_free(struct hrpc_proxy_builder *bld)
+{
+    if (!bld)
+        return;
+    free(bld->proxy);
+    free(bld);
+}
+
+void hrpc_proxy_builder_set_protocol(struct hrpc_proxy_builder *bld,
+                                     const char *protocol)
+{
+    struct hrpc_proxy *proxy = bld->proxy;
+
+    if (proxy->protocol) {
+        if (proxy->protocol != OOM_ERROR) {
+            free(proxy->protocol);
+        }
+        proxy->protocol = NULL;
+    }
+    proxy->protocol = strdup(protocol);
+    if (!proxy->protocol) {
+        proxy->protocol = (char*)OOM_ERROR;
+    }
+}
+
+void hrpc_proxy_builder_set_remote(struct hrpc_proxy_builder *bld,
+                                     const struct sockaddr_in *remote)
+{
+    bld->proxy->call.remote = *remote;
+}
+
+struct hadoop_err *hrpc_proxy_create(struct hrpc_proxy_builder *bld,
+                            struct hrpc_proxy **out)
+{
+    struct hrpc_proxy *proxy;
+    
+    proxy = bld->proxy;
+    free(bld);
+    //fprintf(stderr, "proxy = %p, proxy->protocol = %s, proxy->call.cb = %p\n", proxy, proxy->protocol, proxy->call.cb);
+    if (proxy->call.remote.sin_addr.s_addr == INADDR_ANY) {
+        hrpc_proxy_free(proxy);
+        return hadoop_lerr_alloc(EINVAL, "hrpc_proxy_create: you must specify "
+                                 "a remote.");
+    }
+    if (!proxy->protocol) {
+        hrpc_proxy_free(proxy);
+        return hadoop_lerr_alloc(EINVAL, "hrpc_proxy_create: can't create "
+                                 "a proxy without a protocol argument.");
+    } else if (proxy->protocol == OOM_ERROR) {
+        // There was an OOM error during hrpc_proxy_builder_set_protocol.
+        hrpc_proxy_free(proxy);
+        return hadoop_lerr_alloc(ENOMEM, "hrpc_proxy_create: OOM error.");
+    }
+    *out = proxy;
+    return NULL;
+}
+
+void hrpc_proxy_free(struct hrpc_proxy *proxy)
+{
+    if (!proxy)
+        return;
+    if (hrpc_call_is_active(&proxy->call)) {
+        proxy_log_warn(proxy, "%s", "hrpc_proxy_free: attempt to free a proxy "
+                       "which is currently active!\n");
+        return;
+    }
+    if (proxy->protocol != OOM_ERROR) {
+        free(proxy->protocol);
+    }
+    free(proxy->userdata);
+    free(proxy);
+}
+
+struct hadoop_err *hrpc_proxy_activate(struct hrpc_proxy *proxy)
+{
+    struct hadoop_err *err;
+
+    if (!hrpc_call_activate(&proxy->call)) {
+        err = hadoop_lerr_alloc(EINVAL, "tried to start a call on a "
+                    "proxy which was still in use by another call.");
+        proxy_log_warn(proxy, "hrpc_proxy_activate: %s",
+                       hadoop_err_msg(err));
+        return err;
+    }
+    return NULL;
+}
+
+void hrpc_proxy_deactivate(struct hrpc_proxy *proxy)
+{
+    hrpc_call_deactivate(&proxy->call);
+}
+
+void *hrpc_proxy_alloc_userdata(struct hrpc_proxy *proxy, size_t size)
+{
+    if (size > proxy->userdata_len) {
+        uint8_t *new_userdata = realloc(proxy->userdata, size);
+        if (!new_userdata) {
+            return NULL;
+        }
+        proxy->userdata = new_userdata;
+        proxy->userdata_len = size;
+    }
+    return proxy->userdata;
+}
+
+struct hrpc_sync_ctx *hrpc_proxy_alloc_sync_ctx(struct hrpc_proxy *proxy)
+{
+    struct hrpc_sync_ctx *ctx = 
+        hrpc_proxy_alloc_userdata(proxy, sizeof(struct hrpc_proxy));
+    if (!ctx) {
+        return NULL;
+    }
+    if (uv_sem_init(&ctx->sem, 0)) {
+        return NULL;
+    }
+    memset(&ctx, 0, sizeof(ctx));
+    return ctx;
+}
+
+void hrpc_free_sync_ctx(struct hrpc_sync_ctx *ctx)
+{
+    free(ctx->resp.base);
+    uv_sem_destroy(&ctx->sem);
+}
+
+void hrpc_proxy_sync_cb(struct hrpc_response *resp, struct hadoop_err *err,
+                        void *cb_data)
+{
+    struct hrpc_sync_ctx *ctx = cb_data;
+    ctx->resp = *resp;
+    ctx->err = err;
+    uv_sem_post(&ctx->sem);
+}
+
+void hrpc_proxy_start(struct hrpc_proxy *proxy,
+        const char *method, const void *payload, int payload_packed_len,
+        hrpc_pack_cb_t payload_pack_cb,
+        hrpc_raw_cb_t cb, void *cb_data)
+{
+    RequestHeaderProto req_header = REQUEST_HEADER_PROTO__INIT;
+    uint64_t buf_len;
+    int32_t req_header_len, off = 0;
+    uint8_t *buf;
+    struct hrpc_call *call = &proxy->call;
+
+    call->cb = cb;
+    call->cb_data = cb_data;
+    call->protocol = strdup(proxy->protocol);
+    if (!call->protocol) {
+        hrpc_call_deliver_err(call, hadoop_lerr_alloc(ENOMEM,
+                "hrpc_proxy_start_internal: out of memory"));
+        return;
+    }
+
+    req_header.methodname = (char*)method;
+    req_header.declaringclassprotocolname = proxy->protocol;
+    req_header.clientprotocolversion = 1;
+    req_header_len = request_header_proto__get_packed_size(&req_header);
+    buf_len = varint32_size(req_header_len);
+    buf_len += req_header_len;
+    buf_len += varint32_size(payload_packed_len);
+    buf_len += payload_packed_len;
+    if (buf_len >= MAX_SEND_LEN) {
+        hrpc_call_deliver_err(call,
+            hadoop_lerr_alloc(EINVAL, "hrpc_proxy_setup_header: the "
+                "request length is too long at %"PRId64 " bytes.  The "
+                "maximum we will send is %d bytes.", buf_len, MAX_SEND_LEN));
+        return;
+    }
+    buf = malloc((size_t)buf_len);
+    if (!buf) {
+        hrpc_call_deliver_err(call, 
+            hadoop_lerr_alloc(ENOMEM, "hrpc_proxy_setup_header: "
+                "failed to allocate a buffer of length %"PRId64" bytes.",
+                buf_len));
+        return;
+    }
+    varint32_encode(req_header_len, buf, buf_len, &off);
+    request_header_proto__pack(&req_header, buf + off);
+    off += req_header_len;
+    varint32_encode(payload_packed_len, buf, buf_len, &off);
+    payload_pack_cb(payload, buf + off);
+
+    call->payload = uv_buf_init((char*)buf, buf_len);
+    hrpc_messenger_start_outbound(proxy->msgr, &proxy->call);
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 179 - 0
hadoop-native-core/rpc/proxy.h

@@ -0,0 +1,179 @@
+/**
+ * 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 HADOOP_CORE_RPC_PROXY_H
+#define HADOOP_CORE_RPC_PROXY_H
+
+#include <stdint.h> /* for uint8_t */
+#include <uv.h> /* for uv_buf_t */
+
+struct hadoop_err;
+struct hrpc_messenger;
+struct hrpc_proxy;
+struct hrpc_proxy_builder;
+
+struct hrpc_response {
+    uint8_t *pb_base;
+    int pb_len;
+    void *base;
+};
+
+struct hrpc_sync_ctx {
+    uv_sem_t sem;
+    struct hadoop_err *err;
+    struct hrpc_response resp;
+};
+
+typedef void (*hrpc_raw_cb_t)(struct hrpc_response *,
+    struct hadoop_err *, void *);
+
+typedef size_t (*hrpc_pack_cb_t)(const void *, uint8_t *);
+
+/**
+ * Allocate a Hadoop proxy builder.
+ *
+ * @param msgr      The Hadoop messenger that this proxy will be associated
+ *                    with.
+ * @return          A Hadoop proxy builder, or NULL on OOM.
+ */
+struct hrpc_proxy_builder *hrpc_proxy_builder_alloc(
+                struct hrpc_messenger *msgr);
+
+/**
+ * Free a Hadoop proxy builder.
+ *
+ * @param bld       The Hadoop proxy builder to free.
+ */
+void hrpc_proxy_builder_free(struct hrpc_proxy_builder *bld);
+
+/**
+ * Set the protocol used by a proxy.
+ *
+ * @param bld       The Hadoop proxy builder.
+ * @param proto     The protocol string to use.  Will be deep-copied.
+ */
+void hrpc_proxy_builder_set_protocol(struct hrpc_proxy_builder *bld,
+                                     const char *proto);
+
+/**
+ * Set the remote that the proxy should connect to.
+ *
+ * @param bld       The Hadoop proxy builder.
+ * @param remote    The remote.  Will be deep-copied.
+ */
+void hrpc_proxy_builder_set_remote(struct hrpc_proxy_builder *bld,
+                                   const struct sockaddr_in *remote);
+
+/**
+ * Create a Hadoop proxy
+ *
+ * @param bld       The Hadoop proxy builder to use.
+ *                      The builder will be freed, even on failure.
+ * @param out       (out param) On success, the Hadoop proxy.
+ *
+ * @return          On success, NULL.  On error, the error message.
+ */
+struct hadoop_err *hrpc_proxy_create(struct hrpc_proxy_builder *bld,
+                            struct hrpc_proxy **out);
+
+/**
+ * Free a Hadoop proxy.
+ *
+ * @param proxy     The Hadoop proxy to free.  You must not attempt to free a
+ *                      proxy with a call in progress.
+ */
+void hrpc_proxy_free(struct hrpc_proxy *proxy);
+
+/**
+ * Mark the proxy as active.
+ *
+ * @param proxy                 The proxy
+ *
+ * @return                      NULL on success.  If the proxy is already
+ *                              active, an error will be returned. 
+ */
+struct hadoop_err *hrpc_proxy_activate(struct hrpc_proxy *proxy);
+
+/**
+ * Mark the proxy as inactive.
+ *
+ * This function should not be called after hrpc_proxy_start, since a proxy
+ * that has been started will mark itself as inactive when appropriate.
+ *
+ * @param proxy                 The proxy.
+ */
+void hrpc_proxy_deactivate(struct hrpc_proxy *proxy);
+
+/**
+ * Allocate some data in the proxy's userdata area.
+ *
+ * This will overwrite anything previously allocated in the proxy's userdata
+ * area.  It is not necessary to free this memory later; it will be freed when
+ * the proxy is freed.
+ *
+ * @param proxy                 The proxy
+ *
+ * @return                      NULL on OOM; a pointer to the userdata
+ *                              otherwise.
+ */
+void *hrpc_proxy_alloc_userdata(struct hrpc_proxy *proxy, size_t size);
+
+/**
+ * Allocate a sync context from a proxy via hrpc_proxy_alloc_userdata.
+ *
+ * @param proxy                 The proxy
+ *
+ * @return                      NULL on OOM; the sync context otherwise.
+ */
+struct hrpc_sync_ctx *hrpc_proxy_alloc_sync_ctx(struct hrpc_proxy *proxy);
+
+/**
+ * Free a sync context allocated from a proxy.
+ *
+ * @param proxy                 The sync context.
+ */
+void hrpc_free_sync_ctx(struct hrpc_sync_ctx *ctx);
+
+/**
+ * A callback which synchronous RPCs can use.
+ */
+void hrpc_proxy_sync_cb(struct hrpc_response *resp, struct hadoop_err *err,
+                        void *cb_data);
+
+/**
+ * Start an outgoing RPC from the proxy.
+ *
+ * This method will return after queuing up the RPC to be sent.
+ *
+ * @param proxy                 The Hadoop proxy to use.  A single proxy can
+ *                                  only make one call at once.
+ * @param method                The method we're calling.
+ * @param payload               The protobuf message we're sending.
+ * @param payload_packed_len    Length of payload when serialized.
+ * @param payload_pack_cb       Function used to pack the payload.
+ * @param cb                    Callback invoked when the message is done.
+ * @param cb_data               Data provided along with cb.
+ */
+void hrpc_proxy_start(struct hrpc_proxy *proxy,
+        const char *method, const void *payload, int payload_packed_len,
+        hrpc_pack_cb_t payload_pack_cb,
+        hrpc_raw_cb_t cb, void *cb_data);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 252 - 0
hadoop-native-core/rpc/reactor.c

@@ -0,0 +1,252 @@
+/**
+ * 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 "common/hadoop_err.h"
+#include "common/net.h"
+#include "common/queue.h"
+#include "common/tree.h"
+#include "rpc/call.h"
+#include "rpc/messenger.h"
+#include "rpc/reactor.h"
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <uv.h>
+
+#define reactor_log_warn(reactor, fmt, ...) \
+    fprintf(stderr, "WARN: reactor %p: " fmt, reactor, __VA_ARGS__)
+#define reactor_log_info(msgr, fmt, ...) \
+    fprintf(stderr, "INFO: reactor %p: " fmt, reactor, __VA_ARGS__)
+#define reactor_log_debug(msgr, fmt, ...) \
+    fprintf(stderr, "DBUG: reactor %p: " fmt, reactor, __VA_ARGS__)
+
+RB_GENERATE(hrpc_conns, hrpc_conn, entry, hrpc_conn_compare);
+
+static void reactor_thread_run(void *arg)
+{
+    struct hrpc_reactor *reactor = arg;
+    struct hrpc_conn *conn, *conn_tmp;
+
+    reactor_log_debug(reactor, "%s", "reactor thread starting.\n");
+    uv_run(&reactor->loop, UV_RUN_DEFAULT);
+    reactor_log_debug(reactor, "%s", "reactor thread terminating.\n");
+    RB_FOREACH_SAFE(conn, hrpc_conns, &reactor->conns, conn_tmp) {
+        hrpc_conn_destroy(conn, hadoop_lerr_alloc(ESHUTDOWN, 
+            "hrpc_reactor_start_outbound: the reactor is being shut down."));
+    }
+}
+
+/**
+ * Find an idle connection with a given address in the idle connection map.
+ *
+ * @param reactor       The reactor.
+ * @param remote        The remote address to find.
+ */
+static struct hrpc_conn *reuse_idle_conn(struct hrpc_reactor *reactor,
+            const struct sockaddr_in *remote, const struct hrpc_call *call)
+{
+    struct hrpc_conn *conn;
+    struct hrpc_conn exemplar;
+
+    memset(&exemplar, 0, sizeof(exemplar));
+    exemplar.remote = *remote;
+    exemplar.protocol = call->protocol;
+    conn = RB_NFIND(hrpc_conns, &reactor->conns, &exemplar);
+    if (!conn)
+        return NULL;
+    if (hrpc_conn_usable(conn, remote, call->protocol)) {
+        if (conn->writer.state == HRPC_CONN_WRITE_IDLE) {
+            RB_REMOVE(hrpc_conns, &reactor->conns, conn);
+            return conn;
+        }
+    }
+    return NULL;
+}
+
+static void reactor_begin_shutdown(struct hrpc_reactor *reactor,
+                             struct hrpc_calls *pending_calls)
+{
+    struct hrpc_call *call;
+
+    reactor_log_debug(reactor, "%s", "reactor_begin_shutdown\n");
+    STAILQ_FOREACH(call, pending_calls, entry) {
+        hrpc_call_deliver_err(call, hadoop_lerr_alloc(ESHUTDOWN, 
+            "hrpc_reactor_start_outbound: the reactor is being shut down."));
+    }
+    // Note: other callbacks may still run after the libuv loop has been
+    // stopped.  But we won't block for I/O after this point.
+    uv_stop(&reactor->loop);
+}
+
+static void reactor_async_start_outbound(struct hrpc_reactor *reactor,
+                                         struct hrpc_call *call)
+{
+    char remote_str[64] = { 0 };
+    struct hrpc_conn *conn;
+    struct hadoop_err *err;
+
+    conn = reuse_idle_conn(reactor, &call->remote, call);
+    if (conn) {
+        reactor_log_debug(reactor, "start_outbound(remote=%s) assigning to "
+                       "connection %p\n",
+            net_ipv4_name(&call->remote, remote_str, sizeof(remote_str)), conn);
+        hrpc_conn_start_outbound(conn, call);
+    } else {
+        err = hrpc_conn_create_outbound(reactor, call, &conn);
+        if (err) {
+            reactor_log_warn(reactor, "reactor_async_start_outbound("
+                "remote=%s) got error %s\n",
+                net_ipv4_name(&call->remote, remote_str, sizeof(remote_str)),
+                hadoop_err_msg(err));
+            hrpc_call_deliver_err(call, err);
+            return;
+        }
+        reactor_log_debug(reactor, "start_outbound(remote=%s) created new "
+                       "connection %p\n",
+            net_ipv4_name(&call->remote, remote_str, sizeof(remote_str)), conn);
+    }
+    // Add or re-add the connection to the reactor's tree.
+    RB_INSERT(hrpc_conns, &reactor->conns, conn);
+}
+
+static void reactor_async_cb(uv_async_t *handle)
+{
+    struct hrpc_reactor *reactor = handle->data;
+    int shutdown;
+    struct hrpc_calls pending_calls = STAILQ_HEAD_INITIALIZER(pending_calls);
+    struct hrpc_call *call;
+
+    uv_mutex_lock(&reactor->inbox.lock);
+    shutdown = reactor->inbox.shutdown;
+    STAILQ_SWAP(&reactor->inbox.pending_calls, &pending_calls,
+                hrpc_call);
+    uv_mutex_unlock(&reactor->inbox.lock);
+
+    if (shutdown) {
+        reactor_begin_shutdown(reactor, &pending_calls);
+        return;
+    }
+    STAILQ_FOREACH(call, &pending_calls, entry) {
+        reactor_async_start_outbound(reactor, call);
+    }
+}
+
+void reactor_remove_conn(struct hrpc_reactor *reactor, struct hrpc_conn *conn)
+{
+    struct hrpc_conn *removed;
+
+    removed = RB_REMOVE(hrpc_conns, &reactor->conns, conn);
+    if (!removed) {
+        reactor_log_warn(reactor, "reactor_remove_conn("
+            "conn=%p): no such connection found.\n", conn);
+    }
+}
+
+struct hadoop_err *hrpc_reactor_create(struct hrpc_reactor **out)
+{
+    struct hrpc_reactor *reactor = NULL;
+    struct hadoop_err *err = NULL;
+    int res;
+
+    reactor = calloc(1, sizeof(struct hrpc_reactor));
+    if (!reactor) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_reactor_create: OOM while allocating "
+                                "reactor structure.");
+        goto error_free_reactor;
+    }
+    if (uv_mutex_init(&reactor->inbox.lock) < 0) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_reactor_create: failed to init "
+                                "mutex.");
+        goto error_free_reactor;
+    }
+    RB_INIT(&reactor->conns);
+    STAILQ_INIT(&reactor->inbox.pending_calls);
+    if (uv_loop_init(&reactor->loop)) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_reactor_create: uv_loop_init "
+                                "failed.");
+        goto error_free_mutex;
+    }
+    res = uv_async_init(&reactor->loop, &reactor->inbox.notifier,
+                      reactor_async_cb);
+    if (res) {
+        err = hadoop_uverr_alloc(res, "hrpc_reactor_create: "
+                                 "uv_async_init failed.");
+        goto error_close_loop;
+    }
+    reactor->inbox.notifier.data = reactor;
+    res = uv_thread_create(&reactor->thread, reactor_thread_run, reactor);
+    if (res) {
+        err = hadoop_lerr_alloc(ENOMEM, "hrpc_reactor_create: "
+                                "uv_thread_create failed.");
+        goto error_free_async;
+    }
+    *out = reactor;
+    return NULL;
+
+error_free_async:
+    uv_close((uv_handle_t*)&reactor->inbox.notifier, NULL);
+error_close_loop:
+    uv_loop_close(&reactor->loop);
+error_free_mutex:
+    uv_mutex_destroy(&reactor->inbox.lock);
+error_free_reactor:
+    free(reactor);
+    return err;
+}
+
+void hrpc_reactor_shutdown(struct hrpc_reactor *reactor)
+{
+    reactor_log_debug(reactor, "%s", "hrpc_reactor_shutdown\n");
+    uv_mutex_lock(&reactor->inbox.lock);
+    reactor->inbox.shutdown = 1;
+    uv_mutex_unlock(&reactor->inbox.lock);
+    uv_async_send(&reactor->inbox.notifier);
+    uv_thread_join(&reactor->thread);
+}
+
+void hrpc_reactor_free(struct hrpc_reactor *reactor)
+{
+    reactor_log_debug(reactor, "%s", "hrpc_reactor_free\n");
+    uv_loop_close(&reactor->loop);
+    uv_mutex_destroy(&reactor->inbox.lock);
+    free(reactor);
+}
+
+void hrpc_reactor_start_outbound(struct hrpc_reactor *reactor,
+                                 struct hrpc_call *call)
+{
+    int shutdown = 0;
+
+    uv_mutex_lock(&reactor->inbox.lock);
+    shutdown = reactor->inbox.shutdown;
+    if (!shutdown) {
+        STAILQ_INSERT_TAIL(&reactor->inbox.pending_calls, call, entry);
+    }
+    uv_mutex_unlock(&reactor->inbox.lock);
+    if (shutdown) {
+        hrpc_call_deliver_err(call, hadoop_lerr_alloc(ESHUTDOWN, 
+            "hrpc_reactor_start_outbound: can't start call because the "
+            "reactor has been shut down."));
+    } else {
+        uv_async_send(&reactor->inbox.notifier);
+    }
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 141 - 0
hadoop-native-core/rpc/reactor.h

@@ -0,0 +1,141 @@
+/**
+ * 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 HADOOP_CORE_RPC_REACTOR_H
+#define HADOOP_CORE_RPC_REACTOR_H
+
+#include "common/tree.h"
+#include "rpc/call.h" // for hrpc_call
+#include "rpc/conn.h" // for hrpc_conn_compare
+
+#include <stdint.h>
+#include <uv.h>
+
+/**
+ * The Hadoop reactor thread header.
+ *
+ * Note: this is an internal header which users of the RPC layer don't need to
+ * include.
+ */
+
+RB_HEAD(hrpc_conns, hrpc_conn);
+RB_PROTOTYPE(hrpc_conns, hrpc_conn, entry, hrpc_conn_compare);
+
+struct hrpc_reactor_inbox {
+    /**
+     * Lock which protects the inbox.
+     */
+    uv_mutex_t lock;
+
+    /**
+     * Non-zero if the reactor should shut down.
+     */
+    int shutdown;
+
+    /**
+     * Calls which we have been asked to make.
+     */
+    struct hrpc_calls pending_calls;
+
+    /**
+     * Used to trigger the inbox callback on the reactor thread.
+     *
+     * You do not need the inbox lock to send an async signal.
+     */
+    uv_async_t notifier;
+};
+
+/**
+ * A Hadoop RPC reactor thread.
+ *
+ * Each reactor thread uses libuv to send and receive on multiple TCP sockets
+ * asynchronously.
+ *
+ * With the exception of the inbox, everything in this structure must be
+ * accessed ONLY from the reactor thread.  Nothing is safe to read or write
+ * from another thread except the inbox.
+ */
+struct hrpc_reactor {
+    /**
+     * The inbox for incoming work for this reactor thread.
+     */
+    struct hrpc_reactor_inbox inbox;
+
+    /**
+     * A red-black tree of connections.  This makes it possible to find a
+     * connection to a given IP address quickly.
+     *
+     * We may have multiple connections for the same IP:port combination.
+     */
+    struct hrpc_conns conns;
+
+    /**
+     * The libuv loop.
+     */
+    uv_loop_t loop;
+
+    /**
+     * The libuv timer.  Used to expire connections after a timeout has
+     * elapsed.
+     */
+    uv_timer_t timer;
+
+    /**
+     * The reactor thread.  All reactor callbacks are made from this context.
+     */
+    uv_thread_t thread;
+};
+
+/**
+ * Remove a connection from the reactor.
+ *
+ * @param reactor       The reactor.
+ * @param conn          The connection.
+ */
+void reactor_remove_conn(struct hrpc_reactor *reactor, struct hrpc_conn *conn);
+
+/**
+ * Create the reactor thread.
+ */
+struct hadoop_err *hrpc_reactor_create(struct hrpc_reactor **out);
+
+/**
+ * Shut down the reactor thread and wait for it to terminate.
+ *
+ * All pending calls will get timeout errors.
+ */
+void hrpc_reactor_shutdown(struct hrpc_reactor *reactor);
+
+/**
+ * Free the reactor.
+ */
+void hrpc_reactor_free(struct hrpc_reactor *reactor);
+
+/**
+ * Start an outbound transfer.
+ *
+ * @param reactor       The reactor.
+ * @param conn          The connection.  This connection must be either new, or 
+ * All pending calls will get timeout errors.
+ */
+void hrpc_reactor_start_outbound(struct hrpc_reactor *reactor,
+                                 struct hrpc_call *call);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et

+ 403 - 0
hadoop-native-core/rpc/shorten.c

@@ -0,0 +1,403 @@
+/**
+ * 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 <ctype.h>
+#include <errno.h>
+#include <limits.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define MAX_LINE_LEN                    16384
+#define IFNDEF                          "#ifndef"
+#define IFNDEF_LEN                      (sizeof(IFNDEF) - 1)
+
+enum parse_state {
+    PARSE_IFNDEF = 0,
+    PARSE_STRUCTS_AND_ENUMS,
+    PARSE_MESSAGES,
+    PARSE_DONE,
+};
+
+#define PROTOBUF_C_END_DECLS_STR "PROTOBUF_C_END_DECLS"
+
+static const char *PARSE_STATE_TERMINATORS[] = {
+    "PROTOBUF_C_BEGIN_DECLS",
+    "/* --- messages --- */",
+    PROTOBUF_C_END_DECLS_STR
+};
+
+static const char *MESSAGE_SUFFIXES[] = {
+    "__INIT",
+    "__get_packed_size",
+    "__pack",
+    "__pack_to_buffer",
+    "__unpack",
+    "__free_unpacked",
+};
+
+#define NUM_MESSAGE_SUFFIXES \
+    (sizeof(MESSAGE_SUFFIXES) / sizeof(MESSAGE_SUFFIXES[0]))
+
+static void add_word(char ***words, size_t *num_words, const char *word)
+{
+    size_t new_num_words;
+    char *nword;
+    char **nwords;
+
+    new_num_words = *num_words + 1;
+    nword = strdup(word);
+    if (!nword) {
+        fprintf(stderr, "failed to allocate memory for %Zd words\n",
+                new_num_words);
+        exit(1);
+    }
+    nwords = realloc(*words, sizeof(char **) * new_num_words);
+    if (!nwords) {
+        fprintf(stderr, "failed to allocate memory for %Zd words\n",
+                new_num_words);
+        free(nword);
+        exit(1);
+    }
+    nwords[new_num_words - 1] = nword;
+    *num_words = new_num_words;
+    *words = nwords;
+}
+
+static int has_suffix(const char *str, const char *suffix)
+{
+    int str_len = strlen(str);
+    int suffix_len = strlen(suffix);
+    if (str_len < suffix_len)
+        return 0;
+    return strcmp(str + str_len - suffix_len, suffix) == 0;
+}
+
+static int has_message_suffix(const char *word)
+{
+    size_t i = 0;
+
+    for (i = 0; i < NUM_MESSAGE_SUFFIXES; i++) {
+        if (has_suffix(word, MESSAGE_SUFFIXES[i]))
+            return 1;
+    }
+    return 0;
+}
+
+static void add_words(char ***words, size_t *num_words,
+                    char *line, enum parse_state state)
+{
+    char *word, *ptr = NULL;
+
+    for (word = strtok_r(line, " ", &ptr); word; 
+             word = strtok_r(NULL, " ", &ptr)) {
+        if (word[0] == '_')
+            continue;
+        if (!strstr(word, "__"))
+            continue;
+        if ((state == PARSE_MESSAGES) && (!has_message_suffix(word)))
+            continue;
+        add_word(words, num_words, word);
+    }
+}
+
+static int compare_strings(const void *a, const void *b)
+{
+    return strcmp(*(char * const*)a, *(char * const*)b);
+}
+
+static char *get_last_occurrence(char *haystack, const char *needle)
+{
+    char *val = NULL, *nval;
+    int needle_len = strlen(needle);
+
+    while (1) {
+        nval = strstr(haystack, needle);
+        if (!nval)
+            return val;
+        val = nval + needle_len;
+        haystack = nval + needle_len;
+    }
+}
+
+static char *get_second_last_occurrence(char *haystack, const char *needle)
+{
+    char *pval = NULL, *val = NULL, *nval;
+    int needle_len = strlen(needle);
+
+    while (1) {
+        nval = strstr(haystack, needle);
+        if (!nval)
+            return pval;
+        pval = val;
+        val = nval + needle_len;
+        haystack = nval + needle_len;
+    }
+}
+
+static int has_camel_case(const char *str)
+{
+    int i, prev_lower = 0;
+
+    for (i = 0; str[i]; i++) {
+        if (isupper(str[i])) {
+            if (prev_lower)
+                return 1;
+        } else if (islower(str[i])) {
+            prev_lower = 1;
+        }
+    }
+    return 0;
+}
+
+static char *get_shortened_occurrence(char *str)
+{
+    char *last, *slast;
+        
+    last = get_last_occurrence(str, "__");
+    slast = get_second_last_occurrence(str, "__");
+
+    last = get_last_occurrence(str, "__");
+    if (!last)
+        return NULL;
+    if ((!has_message_suffix(str)) && 
+            (strstr(last, "_") || has_camel_case(last))) { 
+        // Heuristic: if the last bit of the string after the double underscore
+        // has another underscore inside, or has mixed case, we assume it's
+        // complex enough to use on its own.
+        return last;
+    }
+    // Otherwise, we grab the part of the string after the second-last double
+    // underscore.
+    slast = get_second_last_occurrence(str, "__");
+    return slast ? slast : last;
+}
+
+static int output_shortening_macros(char **words, size_t num_words,
+                                    const char *out_path, FILE *out)
+{
+    size_t i;
+    const char *prev_word = "";
+    const char *shortened;
+
+    for (i = 0; i < num_words; i++) {
+        if (strcmp(prev_word, words[i]) == 0) {
+            // skip words we've already done
+            continue;
+        }
+        prev_word = words[i];
+        shortened = get_shortened_occurrence(words[i]);
+        if (shortened) {
+            if (fprintf(out, "#define %s %s\n", shortened, words[i]) < 0) {
+                fprintf(stderr, "error writing to %s\n", out_path);
+                return EIO;
+            }
+        }
+    }
+    return 0;
+}
+
+/**
+ * Remove newlines from a buffer.
+ *
+ * @param line          The buffer.
+ */
+static void chomp(char *line)
+{
+    while (1) {
+        int len = strlen(line);
+        if (len == 0) {
+            return;
+        }
+        if (line[len - 1] != '\n') {
+            return;
+        }
+        line[len - 1] = '\0';
+    }
+}
+
+/**
+ * Remove most non-alphanumeric characters from a buffer.
+ *
+ * @param line          The buffer.
+ */
+static void asciify(char *line)
+{
+    int i;
+
+    for (i = 0; line[i]; i++) {
+        if ((!isalnum(line[i])) && (line[i] != '_') && (line[i] != '#')) {
+            line[i] = ' ';
+        }
+    }
+}
+
+static const char *base_name(const char *path)
+{
+    const char *base;
+    
+    base = rindex(path, '/');
+    if (!base)
+        return NULL;
+    return base + 1;
+}
+
+static int process_file_lines(const char *in_path, const char *out_path,
+                    FILE *in, FILE *out, char ***words, size_t *num_words)
+{
+    int ret;
+    char header_guard[MAX_LINE_LEN] = { 0 };
+    char line[MAX_LINE_LEN] = { 0 };
+    const char *base = base_name(in_path);
+    enum parse_state state = PARSE_IFNDEF;
+
+    if (!base) {
+        fprintf(stderr, "failed to get basename of %s\n", in_path);
+        return EINVAL;
+    }
+    while (1) {
+        if (!fgets(line, MAX_LINE_LEN - 1, in)) {
+            if (ferror(in)) {
+                ret = errno;
+                fprintf(stderr, "error reading %s: %s (%d)\n",
+                        in_path, strerror(ret), ret);
+                return ret;
+            }
+            fprintf(stderr, "error reading %s: didn't find " 
+                    PROTOBUF_C_END_DECLS_STR, in_path);
+            return EINVAL;
+        }
+        if (strstr(line, PARSE_STATE_TERMINATORS[state])) {
+            state = state + 1;
+            if (state == PARSE_DONE) {
+                break;
+            }
+            continue;
+        }
+        chomp(line);
+        asciify(line);
+        switch (state) {
+        case PARSE_IFNDEF:
+            if (strncmp(line, IFNDEF, IFNDEF_LEN) == 0) {
+                strcpy(header_guard, line + IFNDEF_LEN + 1);
+            }
+            break;
+        default:
+            add_words(words, num_words, line, state);
+            break;
+        }
+    }
+    if (!header_guard[0]) {
+        fprintf(stderr, "failed to find header guard for %s\n", in_path);
+        return EINVAL;
+    }
+    qsort(*words, *num_words, sizeof(char*), compare_strings);
+    fprintf(out, "#ifndef %s_S\n", header_guard);
+    fprintf(out, "#define %s_S\n\n", header_guard);
+    fprintf(out, "#include \"%s\"\n\n", base);
+    ret = output_shortening_macros(*words, *num_words, out_path, out);
+    if (ret)
+        return ret;
+    fprintf(out, "\n#endif\n");
+    return 0;
+}
+
+static int process_file(const char *in_path)
+{
+    char out_path[PATH_MAX] = { 0 };
+    int res, ret = 0;
+    FILE *in = NULL, *out = NULL;
+    char **words = NULL;
+    size_t num_words = 0;
+    size_t i;
+
+    res = snprintf(out_path, PATH_MAX, "%s.s", in_path);
+    if ((res < 0) || (res >= PATH_MAX)) {
+        fprintf(stderr, "snprintf error for %s\n", in_path);
+        ret = EINVAL;
+        goto done;
+    }
+    in = fopen(in_path, "r");
+    if (!in) {
+        ret = errno;
+        fprintf(stderr, "failed to open %s for read: error %s (%d)\n",
+                in_path, strerror(ret), ret);
+        goto done;
+    }
+    out = fopen(out_path, "w");
+    if (!out) {
+        ret = errno;
+        fprintf(stderr, "failed to open %s for write: error %s (%d)\n",
+                out_path, strerror(ret), ret);
+        goto done;
+    }
+    ret = process_file_lines(in_path, out_path, in, out, &words, &num_words);
+    for (i = 0; i < num_words; i++) {
+        free(words[i]);
+    }
+    free(words);
+    if (ret) {
+        goto done;
+    }
+    if (fclose(out)) {
+        ret = errno;
+        perror("fclose error");
+    }
+    out = NULL;
+done:
+    if (in) {
+        fclose(in);
+    }
+    if (out) {
+        fclose(out);
+    }
+    return ret;
+}
+
+static void usage(void)
+{
+        fprintf(stderr,
+"shorten: creates header files with shorter definitions for protobuf-c\n"
+"definitions.  Output files will be written to the same paths as input\n"
+"files, but with a .s extension tacked on.\n"
+"\n"
+"usage: shorten [paths-to-headers]\n");
+}
+
+int main(int argc, char **argv)
+{
+    int i, ret, nproc = 0, rval = EXIT_SUCCESS;
+
+    if (argc < 2) {
+        usage();
+        exit(EXIT_SUCCESS);
+    }
+    for (i = 1; i < argc; i++) {
+        ret = process_file(argv[i]);
+        if (ret) {
+            fprintf(stderr, "error processing %s\n", argv[i]);
+            rval = EXIT_FAILURE;
+        } else {
+            nproc++;
+        }
+    }
+    //fprintf(stderr, "successfully processed %d files\n", nproc);
+    return rval;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 75 - 0
hadoop-native-core/rpc/varint-unit.c

@@ -0,0 +1,75 @@
+/**
+ * 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 "common/test.h"
+#include "rpc/varint.h"
+
+#include <errno.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+static int varint32_round_trip_test(int val)
+{
+    int size;
+    int32_t oval = 0;
+    int32_t off = 0;
+    uint8_t buf[16] = { 0 };
+    const size_t buf_len = sizeof(buf)/sizeof(buf[0]);
+
+    size = varint32_size(val);
+    EXPECT_INT_ZERO(varint32_encode(val, buf, buf_len, &off));
+    EXPECT_INT_EQ(size, off);
+    off = 0;
+    EXPECT_INT_ZERO(varint32_decode(&oval, buf, buf_len, &off));
+    EXPECT_INT_EQ(size, off);
+    EXPECT_INT_EQ(val, oval);
+    return 0;
+}
+
+static int be32_round_trip_test(int val)
+{
+    int32_t oval;
+    uint8_t buf[sizeof(int32_t)] = { 0 };
+
+    be32_encode(val, buf);
+    oval = be32_decode(buf);
+    EXPECT_INT_EQ(val, oval);
+    return 0;
+}
+
+static int round_trip_test(int var)
+{
+    EXPECT_INT_ZERO(varint32_round_trip_test(var));
+    EXPECT_INT_ZERO(be32_round_trip_test(var));
+    return 0;
+}
+
+int main()
+{
+    round_trip_test(0);
+    round_trip_test(123);
+    round_trip_test(6578);
+    round_trip_test(0x7fffffff);
+    round_trip_test(-15);
+    round_trip_test(-1);
+    return EXIT_SUCCESS;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 91 - 0
hadoop-native-core/rpc/varint.c

@@ -0,0 +1,91 @@
+/**
+ * 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 "varint.h"
+
+#include <errno.h>
+#include <stdint.h>
+
+int varint32_size(int32_t val)
+{
+    if ((val & (0xffffffff <<  7)) == 0) return 1;
+    if ((val & (0xffffffff << 14)) == 0) return 2;
+    if ((val & (0xffffffff << 21)) == 0) return 3;
+    if ((val & (0xffffffff << 28)) == 0) return 4;
+    return 5;
+}
+
+int varint32_encode(int32_t val, uint8_t *buf, int32_t len, int32_t *off)
+{
+    int32_t o = *off;
+    uint32_t var = (uint32_t)val;
+
+    while (1) {
+        if (o == len)
+            return -ENOBUFS;
+        if (var <= 127) {
+            buf[o++] = var;
+            break;
+        }
+        buf[o++] = 0x80 | (var & 0x7f);
+        var >>= 7;
+    }
+    *off = o;
+    return 0;
+}
+
+int varint32_decode(int32_t *val, const uint8_t *buf, int32_t len,
+                    int32_t *off)
+{
+    uint32_t accum = 0;
+    int shift = 0, o = *off, idx = 0;
+    uint8_t b;
+
+    do {
+        if (o == len)
+            return -ENOBUFS;
+        if (idx++ > 5)
+            return -EINVAL;
+        b = buf[o++];
+        accum += (b & 0x7f) << shift;
+        shift += 7;
+    } while(b & 0x80);
+    *val = (int32_t)accum;
+    *off = o;
+    return 0;
+}
+
+void be32_encode(int32_t val, uint8_t *buf)
+{
+    buf[0] = (val >> 24) & 0xff;
+    buf[1] = (val >> 16) & 0xff;
+    buf[2] = (val >> 8) & 0xff;
+    buf[3] = (val >> 0) & 0xff;
+}
+
+int32_t be32_decode(const uint8_t *buf)
+{
+    int32_t v = 0;
+    v |= (buf[0] << 24);
+    v |= (buf[1] << 16);
+    v |= (buf[2] << 8);
+    v |= (buf[3] << 0);
+    return v;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 82 - 0
hadoop-native-core/rpc/varint.h

@@ -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.
+ */
+
+#ifndef HADOOP_CORE_VARINT_H
+#define HADOOP_CORE_VARINT_H
+
+#include <stdint.h>
+
+/**
+ * Compute the size of a varint.
+ *
+ * @param val           The value to encode.
+ *
+ * @return              The number of bytes it will take to encode.
+ */
+int varint32_size(int32_t val);
+
+/**
+ * Encode a 32-bit varint.
+ *
+ * @param val           The value to encode.
+ * @param buf           The buffer to encode into.
+ * @param len           The length of the buffer.
+ * @param off           (inout) The offset to start writing at.  This will be
+ *                      updated with the new offset after we finish the
+ *                      encoding.
+ *
+ * @return              0 on success; -EINVAL if we ran out of space.
+ */
+int varint32_encode(int32_t val, uint8_t *buf, int32_t max, int32_t *off);
+
+/**
+ * Decode a 32-bit varint.
+ *
+ * @param val           (out param) The decoded value.
+ * @param buf           The buffer to decode from.
+ * @param len           The length of the buffer.
+ * @param off           (inout) The offset to start reading at.  This will be
+ *                      updated with the new offset after we finish the
+ *                      decoding.
+ *
+ * @return              0 on success; -EINVAL if we ran out of space.
+ */
+int varint32_decode(int32_t *val, const uint8_t *buf, int32_t max,
+                    int32_t *off);
+
+/**
+ * Encode a fixed-len, big-endian int32_t.
+ *
+ * @param val           The value to encode.
+ * @param buf           The buffer to encode into.
+ */
+void be32_encode(int32_t val, uint8_t *buf);
+
+/**
+ * Decode a fixed-len, big-endian int32_t.
+ *
+ * @param buf           The buffer to decode from.  The buffer must be at
+ *                        least 4 bytes long.
+ *
+ * @return              The decoded integer.
+ */
+int32_t be32_decode(const uint8_t *buf);
+
+#endif
+
+// vim: ts=4:sw=4:tw=79:et